diff --git a/deps/oblib/src/lib/ob_name_id_def.h b/deps/oblib/src/lib/ob_name_id_def.h index b39d9f1234..8c2bbcb943 100644 --- a/deps/oblib/src/lib/ob_name_id_def.h +++ b/deps/oblib/src/lib/ob_name_id_def.h @@ -224,6 +224,7 @@ DEF_NAME(id, "id") DEF_NAME(log_pending_cb, "log pending callback") DEF_NAME(submit_log, "submit log") DEF_NAME(submit_instant_log, "submit instant log") + DEF_NAME(submit_log_for_freeze, "submit log for freeze") DEF_NAME(submit_rollback_log, "submit rollback log") DEF_NAME(retry_submit_log, "retry submit log") DEF_NAME(submit_log_cb, "submit log callback") @@ -282,6 +283,7 @@ DEF_NAME(id, "id") DEF_NAME(release_snapshot, "release snapshot") DEF_NAME(start_epoch, "start epoch") DEF_NAME(create_implicit_savepoint, "create implicit savepoint") + DEF_NAME(create_branch_savepoint, "create branch savepoint") DEF_NAME(create_local_implicit_savepoint, "create local implicit savepoint") DEF_NAME(create_global_implicit_savepoint, "create global implicit savepoint") DEF_NAME(rollback_implicit_savepoint, "rollback implicit savepoint") @@ -691,6 +693,7 @@ DEF_NAME(id, "id") DEF_NAME(rollback_from, "rollback_from") DEF_NAME(gtrid, "global transaction id") DEF_NAME(bqual, "branch qualifier") + DEF_NAME(branch, "branch") DEF_NAME(is_stmt, "is normal dml stmt") DEF_NAME(xa_flag, "flag for xa transaciton") DEF_NAME(remote_pull, "remote pull operation") diff --git a/deps/oblib/src/lib/trace/ob_trace_event.h b/deps/oblib/src/lib/trace/ob_trace_event.h index 1f3f8426f8..2786229a0f 100644 --- a/deps/oblib/src/lib/trace/ob_trace_event.h +++ b/deps/oblib/src/lib/trace/ob_trace_event.h @@ -91,20 +91,20 @@ public: } return pos; } - void check_lock() + void check_lock() const { if (need_lock_) { (void)lock_.lock(); } } - void check_unlock() + void check_unlock() const { if (need_lock_) { (void)lock_.unlock(); } } private: - lib::ObMutex lock_; + mutable lib::ObMutex lock_; bool need_lock_; }; diff --git a/deps/oblib/src/lib/utility/ob_tracepoint.h b/deps/oblib/src/lib/utility/ob_tracepoint.h index ce3d1851dd..e93dd6412a 100644 --- a/deps/oblib/src/lib/utility/ob_tracepoint.h +++ b/deps/oblib/src/lib/utility/ob_tracepoint.h @@ -775,6 +775,7 @@ class EventTable EN_TX_FREE_ROUTE_STATE_SIZE = 2003, // Transaction common EN_TX_RESULT_INCOMPLETE = 2011, + EN_CHECK_TX_CTX_LOCK = 2013, EN_THREAD_HANG = 2022, EN_ENABLE_SET_TRACE_CONTROL_INFO = 2100, diff --git a/mittest/mtlenv/storage/test_memtable_v2.cpp b/mittest/mtlenv/storage/test_memtable_v2.cpp index 6cbc3a0323..173a52b73e 100644 --- a/mittest/mtlenv/storage/test_memtable_v2.cpp +++ b/mittest/mtlenv/storage/test_memtable_v2.cpp @@ -167,9 +167,12 @@ public: is_sstable_contains_lock_ = false; // mock master key getter - ASSERT_EQ(OB_SUCCESS, ObMasterKeyGetter::instance().init(NULL)); + ObMasterKeyGetter::instance().init(NULL); - TRANS_LOG(INFO, "setup success"); + const testing::TestInfo* const test_info = + testing::UnitTest::GetInstance()->current_test_info(); + const char * test_name = test_info->name(); + _TRANS_LOG(INFO, ">>> setup success : %s", test_name); } virtual void TearDown() override @@ -189,8 +192,7 @@ public: allocator_.reset(); allocator2_.reset(); - ObMasterKeyGetter::instance().destroy(); - + ObMasterKeyGetter::instance().stop(); TRANS_LOG(INFO, "teardown success"); } @@ -281,6 +283,45 @@ public: } } + int flush_txn_log(ObStoreCtx *store_ctx, const int64_t scn, const int cnt, ObCallbackScope &scope) + { + ObMemtableCtx *mt_ctx = store_ctx->mvcc_acc_ctx_.mem_ctx_; + ObTxCallbackList &cb_list = mt_ctx->trans_mgr_.callback_list_; + ObMvccRowCallback *next = NULL; + ObIMemtable* mt; + share::SCN scn_x; + int ret = scn_x.convert_for_tx(scn); + for (ObMvccRowCallback *iter = (ObMvccRowCallback *)(cb_list.get_log_cursor()); + iter != (ObMvccRowCallback *)(cb_list.get_guard()); iter = next) { + next = (ObMvccRowCallback *)(iter->get_next()); + iter->log_submitted_cb(scn_x, mt); + if (!*scope.start_) { scope.start_ = iter; } + scope.end_ = iter; + scope.host_ = &cb_list; + scope.cnt_ +=1; + if (scope.cnt_ == cnt) { + break; + } + } + cb_list.submit_log_succ(scope); + return ret; + } + + int sync_txn_log_fail(ObStoreCtx *store_ctx, + ObCallbackScope &scope, + const int64_t palf_applied_scn, + int64_t &removed_cnt) + { + share::SCN scn_x; + int ret = OB_SUCCESS; + if (OB_SUCC(scn_x.convert_for_tx(palf_applied_scn))) { + ObMemtableCtx *mt_ctx = store_ctx->mvcc_acc_ctx_.mem_ctx_; + ObTxCallbackList &cb_list = mt_ctx->trans_mgr_.callback_list_; + ret = cb_list.sync_log_fail(scope, scn_x, removed_cnt); + } + return ret; + } + void prepare_txn(ObStoreCtx *store_ctx, const int64_t prepare_version) { @@ -299,13 +340,22 @@ public: { share::SCN commit_scn; commit_scn.convert_for_tx(commit_version); + ObIMemtable *mt; ObPartTransCtx *tx_ctx = store_ctx->mvcc_acc_ctx_.tx_ctx_; ObMemtableCtx *mt_ctx = store_ctx->mvcc_acc_ctx_.mem_ctx_; tx_ctx->exec_info_.state_ = ObTxState::COMMIT; tx_ctx->ctx_tx_data_.set_commit_version(commit_scn); tx_ctx->ctx_tx_data_.set_state(ObTxData::COMMIT); - if (need_write_back) { + // flush log for all txNode + { + ObTxCallbackList &cb_list = mt_ctx->trans_mgr_.callback_list_; + for (ObMvccRowCallback *iter = (ObMvccRowCallback *)(cb_list.get_guard()->get_next()); + iter != (ObMvccRowCallback *)(cb_list.get_guard()); + iter = (ObMvccRowCallback *)(iter->get_next())) { + iter->log_submitted_cb(share::SCN::minus(commit_scn, 10), mt); + } + } EXPECT_EQ(OB_SUCCESS, mt_ctx->trans_end(true, /*commit*/ commit_scn, commit_scn /*commit log ts*/)); @@ -340,7 +390,6 @@ public: } } } - ObStoreCtx *start_tx(const ObTransID &tx_id, const bool for_replay = false) { ObTxDesc *tx_desc = new ObTxDesc(); @@ -387,10 +436,10 @@ public: store_ctx->mvcc_acc_ctx_.type_ = ObMvccAccessCtx::T::WRITE; store_ctx->mvcc_acc_ctx_.snapshot_.tx_id_ = store_ctx->mvcc_acc_ctx_.tx_id_; store_ctx->mvcc_acc_ctx_.snapshot_.version_ = snapshot_scn; - store_ctx->mvcc_acc_ctx_.snapshot_.scn_ = ObTxSEQ(ObSequence::get_max_seq_no(), 0); + store_ctx->mvcc_acc_ctx_.snapshot_.scn_ = ObTxSEQ(ObSequence::get_max_seq_no()); const int64_t abs_expire_time = expire_time + ::oceanbase::common::ObTimeUtility::current_time(); store_ctx->mvcc_acc_ctx_.abs_lock_timeout_ = abs_expire_time; - store_ctx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no(), 0); + store_ctx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no()); } void start_pdml_stmt(ObStoreCtx *store_ctx, const share::SCN snapshot_scn, @@ -404,7 +453,7 @@ public: store_ctx->mvcc_acc_ctx_.snapshot_.scn_ = read_seq_no; const int64_t abs_expire_time = expire_time + ::oceanbase::common::ObTimeUtility::current_time(); store_ctx->mvcc_acc_ctx_.abs_lock_timeout_ = abs_expire_time; - store_ctx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no(), 0); + store_ctx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no()); } void print_callback(ObStoreCtx *wtx) { @@ -442,9 +491,11 @@ public: if (OB_FAIL(context.init(query_flag, *wtx, allocator_, trans_version_range))) { TRANS_LOG(WARN, "Fail to init access context", K(ret)); } - - EXPECT_EQ(expect_ret, (ret = memtable->set(iter_param_, context, columns_, write_row, encrypt_meta_))); - + ret = memtable->set(iter_param_, context, columns_, write_row, encrypt_meta_); + if (ret == -5024) { + TRANS_LOG(ERROR, "nima", K(ret), K(write_row)); + } + EXPECT_EQ(expect_ret, ret); TRANS_LOG(INFO, "======================= end write tx ======================", K(ret), K(wtx->mvcc_acc_ctx_.tx_id_), K(*wtx), K(snapshot), K(expire_time), K(write_row)); } @@ -752,13 +803,16 @@ public: ObIMemtableCtx *mem_ctx = store_ctx->mvcc_acc_ctx_.mem_ctx_; char *redo_log_buffer = new char[REDO_BUFFER_SIZE]; encrypt_info.init(); - EXPECT_EQ(OB_SUCCESS, mem_ctx->fill_redo_log(redo_log_buffer, - REDO_BUFFER_SIZE, - serialize_pos, - helper)); + ObTxFillRedoCtx ctx; + ctx.buf_ = redo_log_buffer; + ctx.buf_len_ = REDO_BUFFER_SIZE; + ctx.buf_pos_ = serialize_pos; + ctx.helper_ = &helper; + ctx.fill_count_ = 0; + EXPECT_EQ(OB_SUCCESS, mem_ctx->fill_redo_log(ctx)); EXPECT_EQ(OB_SUCCESS, mmi.deserialize(redo_log_buffer, - serialize_pos, + ctx.buf_pos_, deserialize_pos, encrypt_info)); } @@ -770,19 +824,21 @@ public: int64_t pos = 0; ObRedoLogSubmitHelper helper; ObPartTransCtx *tx_ctx = store_ctx->mvcc_acc_ctx_.tx_ctx_; - ObTxRedoLog redo_log(1000 /*fake log_no*/, - 1000 /*fake cluster_version_*/); + ObTxRedoLog redo_log(1000 /*fake cluster_version_*/); redo_log.set_mutator_buf(redo_log_buffer); redo_log.set_mutator_size(REDO_BUFFER_SIZE, false /*after_fill*/); ObIMemtableCtx *mem_ctx = store_ctx->mvcc_acc_ctx_.mem_ctx_; - EXPECT_EQ(OB_SUCCESS, mem_ctx->fill_redo_log(redo_log.get_mutator_buf(), - redo_log.get_mutator_size(), - mutator_size, - helper)); + ObTxFillRedoCtx ctx; + ctx.buf_ = redo_log.get_mutator_buf(); + ctx.buf_len_ = redo_log.get_mutator_size(); + ctx.buf_pos_ = mutator_size; + ctx.helper_ = &helper; + ctx.fill_count_ = 0; + EXPECT_EQ(OB_SUCCESS, mem_ctx->fill_redo_log(ctx)); - redo_log.set_mutator_size(mutator_size, true /*after_fill*/); + redo_log.set_mutator_size(ctx.buf_pos_, true /*after_fill*/); EXPECT_EQ(OB_SUCCESS, redo_log.serialize(redo_log_buffer, REDO_BUFFER_SIZE, pos)); } @@ -818,6 +874,7 @@ public: OB_MAX_MASTER_KEY_LENGTH, master_key_len)); meta.master_key_.get_content().set_length(master_key_len); + TRANS_LOG(INFO, "deserialized master key", K(meta.master_key_)); EXPECT_EQ(OB_SUCCESS, ObBlockCipher::decrypt(meta.master_key_.ptr(), meta.master_key_.size(), meta.encrypted_table_key_.ptr(), @@ -828,7 +885,7 @@ public: decrypted_table_key, out_len)); meta.table_key_.set_content(decrypted_table_key, out_len); - + TRANS_LOG(INFO, "deserialized table_key", K(meta.table_key_)); EXPECT_EQ(true, meta.is_valid()); } else { ob_abort(); @@ -857,8 +914,8 @@ public: K(*store_ctx)); share::SCN replay_scn; replay_scn.convert_for_tx(replay_log_ts); - store_ctx->mvcc_acc_ctx_.mem_ctx_->set_redo_scn(replay_scn); EXPECT_EQ(OB_SUCCESS, memtable->replay_row(*store_ctx, + replay_scn, &mmi)); } } @@ -884,8 +941,8 @@ public: K(*store_ctx)); share::SCN replay_scn; replay_scn.convert_for_tx(replay_log_ts); - store_ctx->mvcc_acc_ctx_.mem_ctx_->set_redo_scn(replay_scn); EXPECT_EQ(OB_SUCCESS, memtable->replay_row(*store_ctx, + replay_scn, &mmi)); } } @@ -933,7 +990,6 @@ public: EXPECT_EQ(mt, cb->memtable_); EXPECT_EQ(scn, cb->scn_); EXPECT_EQ(is_link, cb->is_link_); - EXPECT_EQ(need_fill_redo, cb->need_fill_redo_); EXPECT_EQ(need_fill_redo, cb->need_submit_log_); EXPECT_EQ(seq_no, cb->seq_no_); ObStoreRowkey *rowkey = cb->key_.rowkey_; @@ -961,7 +1017,7 @@ public: const uint8_t ndt_type = NDT_NORMAL) { ASSERT_NE(NULL, (long)tnode); - TRANS_LOG(INFO, "=============== VERIFY TRANS NODE START ===============", K(*tnode)); + TRANS_LOG(INFO, "=============== VERIFY TRANS NODE START ===============", K(*tnode), K(log_ts)); share::SCN scn; scn.convert_for_tx(log_ts); @@ -1026,7 +1082,6 @@ public: EXPECT_EQ(wmt, cb->memtable_); EXPECT_EQ(share::SCN::max_scn(), cb->scn_); EXPECT_EQ(true, cb->is_link_); - EXPECT_EQ(true, cb->need_fill_redo_); EXPECT_EQ(true, cb->need_submit_log_); EXPECT_EQ(seq_no, cb->seq_no_); ObStoreRowkey *rowkey = cb->key_.rowkey_; @@ -1195,7 +1250,7 @@ TEST_F(TestMemtableV2, test_write_read_conflict) memtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); verify_cb(get_tx_last_cb(wtx), memtable, @@ -1310,7 +1365,8 @@ TEST_F(TestMemtableV2, test_write_read_conflict) ObMvccTransNode::F_COMMITTED | ObMvccTransNode::F_DELAYED_CLEANOUT, ObDmlFlag::DF_INSERT, 1, /*key*/ - 2 /*value*/); + 2 /*value*/, + 2000 - 10); verify_mvcc_row(tmp_row, ObDmlFlag::DF_INSERT, ObDmlFlag::DF_INSERT, @@ -1338,7 +1394,7 @@ TEST_F(TestMemtableV2, test_tx_abort) memtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); verify_cb(get_tx_last_cb(wtx), memtable, @@ -1454,7 +1510,7 @@ TEST_F(TestMemtableV2, test_write_write_conflict) verify_cb(get_tx_last_cb(wtx), memtable, - ObTxSEQ(ObSequence::get_max_seq_no(),0), + ObTxSEQ(ObSequence::get_max_seq_no()), 1, /*key*/ true /*is_link*/); verify_tnode(get_tx_last_tnode(wtx), @@ -1463,7 +1519,7 @@ TEST_F(TestMemtableV2, test_write_write_conflict) memtable, wtx->mvcc_acc_ctx_.tx_id_, INT64_MAX, /*trans_version*/ - ObTxSEQ(ObSequence::get_max_seq_no(),0), + ObTxSEQ(ObSequence::get_max_seq_no()), 0, /*modify_count*/ ObMvccTransNode::F_INIT, ObDmlFlag::DF_INSERT, @@ -1521,7 +1577,7 @@ TEST_F(TestMemtableV2, test_write_write_conflict) verify_cb(get_tx_last_cb(wtx), memtable, - ObTxSEQ(ObSequence::get_max_seq_no(),0), + ObTxSEQ(ObSequence::get_max_seq_no()), 1, /*key*/ true /*is_link*/); verify_tnode(get_tx_last_tnode(wtx), @@ -1530,7 +1586,7 @@ TEST_F(TestMemtableV2, test_write_write_conflict) memtable, wtx->mvcc_acc_ctx_.tx_id_, INT64_MAX, /*trans_version*/ - ObTxSEQ(ObSequence::get_max_seq_no(),0), + ObTxSEQ(ObSequence::get_max_seq_no()), 1, /*modify_count*/ ObMvccTransNode::F_INIT, ObDmlFlag::DF_INSERT, @@ -1585,7 +1641,7 @@ TEST_F(TestMemtableV2, test_write_write_conflict) memtable, 2100, /*snapshot version*/ write_row2); - const auto wtx2_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx2_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); verify_cb(get_tx_last_cb(wtx2), memtable, @@ -1667,7 +1723,7 @@ TEST_F(TestMemtableV2, test_lock) memtable, 1000, /*snapshot version*/ rowkey); - const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); verify_cb(get_tx_last_cb(ltx), memtable, @@ -1775,7 +1831,7 @@ TEST_F(TestMemtableV2, test_lock) memtable, 2500, /*snapshot version*/ rowkey); - const auto wtx2_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx2_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); verify_cb(get_tx_last_cb(wtx2), memtable, wtx2_seq_no, @@ -1840,7 +1896,7 @@ TEST_F(TestMemtableV2, test_lock) memtable, 4500, /*snapshot version*/ write_row3); - const auto wtx3_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx3_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); verify_cb(get_tx_last_cb(wtx3), memtable, wtx3_seq_no, @@ -1967,12 +2023,12 @@ TEST_F(TestMemtableV2, test_rollback_to) memtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); write_tx(wtx, memtable, 1000, /*snapshot version*/ write_row2); - const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no()); print_callback(wtx); @@ -2066,7 +2122,7 @@ TEST_F(TestMemtableV2, test_replay) lmemtable, 500, /*snapshot version*/ write_row3); - const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); commit_txn(wtx3, 800,/*commit_version*/ false/*need_write_back*/); @@ -2077,12 +2133,12 @@ TEST_F(TestMemtableV2, test_replay) lmemtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); write_tx(wtx, lmemtable, 1200, /*snapshot version*/ write_row2); - const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMemtableMutatorIterator mmi; mock_replay_iterator(wtx, mmi); @@ -2221,140 +2277,140 @@ TEST_F(TestMemtableV2, test_replay) fmemtable->destroy(); } -TEST_F(TestMemtableV2, test_replay_with_clog_encryption) -{ - ObMemtable *lmemtable = create_memtable(); - ObMemtable *fmemtable = create_memtable(); +// TEST_F(TestMemtableV2, test_replay_with_clog_encryption) +// { +// ObMemtable *lmemtable = create_memtable(); +// ObMemtable *fmemtable = create_memtable(); - TRANS_LOG(INFO, "######## CASE1: txn1 write row in lmemtable"); - ObDatumRowkey rowkey; - ObStoreRow write_row; - ObDatumRowkey rowkey2; - ObStoreRow write_row2; +// TRANS_LOG(INFO, "######## CASE1: txn1 write row in lmemtable"); +// ObDatumRowkey rowkey; +// ObStoreRow write_row; +// ObDatumRowkey rowkey2; +// ObStoreRow write_row2; - EXPECT_EQ(OB_SUCCESS, mock_row(1, /*key*/ - 2, /*value*/ - rowkey, - write_row)); - EXPECT_EQ(OB_SUCCESS, mock_row(1, /*key*/ - 3, /*value*/ - rowkey2, - write_row2)); +// EXPECT_EQ(OB_SUCCESS, mock_row(1, /*key*/ +// 2, /*value*/ +// rowkey, +// write_row)); +// EXPECT_EQ(OB_SUCCESS, mock_row(1, /*key*/ +// 3, /*value*/ +// rowkey2, +// write_row2)); - ObTransID write_tx_id = ObTransID(1); - ObStoreCtx *wtx = start_tx(write_tx_id); +// ObTransID write_tx_id = ObTransID(1); +// ObStoreCtx *wtx = start_tx(write_tx_id); - //create encrypt_meta and store encrypt_meta in encrypt_meta_ - ObSerializeEncryptMeta encrypt_meta; - create_and_store_encrypt_info(encrypt_meta); +// //create encrypt_meta and store encrypt_meta in encrypt_meta_ +// ObSerializeEncryptMeta encrypt_meta; +// create_and_store_encrypt_info(encrypt_meta); - //encrypt_meta_ will be added to memtable during ObMemtable::set - write_tx(wtx, - lmemtable, - 1000, /*snapshot version*/ - write_row); - const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(), 0); - write_tx(wtx, - lmemtable, - 1200, /*snapshot version*/ - write_row2); - const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no(), 0); +// //encrypt_meta_ will be added to memtable during ObMemtable::set +// write_tx(wtx, +// lmemtable, +// 1000, /*snapshot version*/ +// write_row); +// const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); +// write_tx(wtx, +// lmemtable, +// 1200, /*snapshot version*/ +// write_row2); +// const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no()); - //use encrypt_meta_ in memtable during submitting log - char *redo_log_buffer = new char[REDO_BUFFER_SIZE]; - serialize_encrypted_redo_log(wtx, redo_log_buffer); +// //use encrypt_meta_ in memtable during submitting log +// char *redo_log_buffer = new char[REDO_BUFFER_SIZE]; +// serialize_encrypted_redo_log(wtx, redo_log_buffer); - commit_txn(wtx, - 2000,/*commit_version*/ - false/*need_write_back*/); +// commit_txn(wtx, +// 2000,/*commit_version*/ +// false/*need_write_back*/); - TRANS_LOG(INFO, "######## CASE2: txn2 replay row in fmemtable"); +// TRANS_LOG(INFO, "######## CASE2: txn2 replay row in fmemtable"); - ObTxRedoLogTempRef temp_ref; - ObTxRedoLog redo_log(temp_ref); +// ObTxRedoLogTempRef temp_ref; +// ObTxRedoLog redo_log(temp_ref); - //get encrypt_info during deserializing log - ObMemtableMutatorIterator mmi; - ObCLogEncryptInfo encrypt_info; - encrypt_info.init(); - deserialize_redo_log_extract_encryption(redo_log_buffer, redo_log, mmi, encrypt_info); +// //get encrypt_info during deserializing log +// ObMemtableMutatorIterator mmi; +// ObCLogEncryptInfo encrypt_info; +// encrypt_info.init(); +// deserialize_redo_log_extract_encryption(redo_log_buffer, redo_log, mmi, encrypt_info); - ObTransID replay_tx_id = ObTransID(2); - ObStoreCtx *ptx = start_tx(replay_tx_id, true); - replay_tx_with_encryption(ptx, - fmemtable, - 1300, /*replay_log_ts*/ - mmi, - encrypt_info); - read_row(ptx, - fmemtable, - rowkey, - 1500, /*snapshot version*/ - 1, /*key*/ - 3 /*value*/); +// ObTransID replay_tx_id = ObTransID(2); +// ObStoreCtx *ptx = start_tx(replay_tx_id, true); +// replay_tx_with_encryption(ptx, +// fmemtable, +// 1300, /*replay_log_ts*/ +// mmi, +// encrypt_info); +// read_row(ptx, +// fmemtable, +// rowkey, +// 1500, /*snapshot version*/ +// 1, /*key*/ +// 3 /*value*/); - ObMvccRowCallback *first_cb = (ObMvccRowCallback *)(get_tx_last_cb(ptx)->prev_); - verify_cb(get_tx_last_cb(ptx), - fmemtable, - wtx_seq_no2, - 1, /*key*/ - true, /*is_link*/ - false,/*need_fill_redo*/ - 1300 /*log_ts*/); - verify_cb(first_cb, - fmemtable, - wtx_seq_no1, - 1, /*key*/ - true, /*is_link*/ - false,/*need_fill_redo*/ - 1300 /*log_ts*/); - verify_tnode(get_tx_last_tnode(ptx), - first_cb->tnode_, /*prev tnode*/ - NULL, /*next tnode*/ - lmemtable, - ptx->mvcc_acc_ctx_.tx_id_, - INT64_MAX, /*trans_version*/ - wtx_seq_no2, - 1, /*modify_count*/ - ObMvccTransNode::F_INIT, - ObDmlFlag::DF_INSERT, - 1, /*key*/ - 3, /*value*/ - 1300 /*log_ts*/); - verify_tnode(first_cb->tnode_, - NULL, /*prev tnode*/ - get_tx_last_tnode(ptx), /*next tnode*/ - lmemtable, - ptx->mvcc_acc_ctx_.tx_id_, - INT64_MAX, /*trans_version*/ - wtx_seq_no1, - 0, /*modify_count*/ - ObMvccTransNode::F_INIT, - ObDmlFlag::DF_INSERT, - 1, /*key*/ - 2, /*value*/ - 1300 /*log_ts*/); - verify_mvcc_row(get_tx_last_mvcc_row(ptx), - ObDmlFlag::DF_NOT_EXIST, - ObDmlFlag::DF_NOT_EXIST, - get_tx_last_tnode(ptx), - 0, /*max_trans_version*/ - 2 /*total_trans_node_cnt*/); - commit_txn(ptx, - 2500,/*commit_version*/ - false/*need_write_back*/); - read_row(fmemtable, - rowkey, - 3000, /*snapshot version*/ - 1, /*key*/ - 3 /*value*/); +// ObMvccRowCallback *first_cb = (ObMvccRowCallback *)(get_tx_last_cb(ptx)->prev_); +// verify_cb(get_tx_last_cb(ptx), +// fmemtable, +// wtx_seq_no2, +// 1, /*key*/ +// true, /*is_link*/ +// false,/*need_fill_redo*/ +// 1300 /*log_ts*/); +// verify_cb(first_cb, +// fmemtable, +// wtx_seq_no1, +// 1, /*key*/ +// true, /*is_link*/ +// false,/*need_fill_redo*/ +// 1300 /*log_ts*/); +// verify_tnode(get_tx_last_tnode(ptx), +// first_cb->tnode_, /*prev tnode*/ +// NULL, /*next tnode*/ +// lmemtable, +// ptx->mvcc_acc_ctx_.tx_id_, +// INT64_MAX, /*trans_version*/ +// wtx_seq_no2, +// 1, /*modify_count*/ +// ObMvccTransNode::F_INIT, +// ObDmlFlag::DF_INSERT, +// 1, /*key*/ +// 3, /*value*/ +// 1300 /*log_ts*/); +// verify_tnode(first_cb->tnode_, +// NULL, /*prev tnode*/ +// get_tx_last_tnode(ptx), /*next tnode*/ +// lmemtable, +// ptx->mvcc_acc_ctx_.tx_id_, +// INT64_MAX, /*trans_version*/ +// wtx_seq_no1, +// 0, /*modify_count*/ +// ObMvccTransNode::F_INIT, +// ObDmlFlag::DF_INSERT, +// 1, /*key*/ +// 2, /*value*/ +// 1300 /*log_ts*/); +// verify_mvcc_row(get_tx_last_mvcc_row(ptx), +// ObDmlFlag::DF_NOT_EXIST, +// ObDmlFlag::DF_NOT_EXIST, +// get_tx_last_tnode(ptx), +// 0, /*max_trans_version*/ +// 2 /*total_trans_node_cnt*/); +// commit_txn(ptx, +// 2500,/*commit_version*/ +// false/*need_write_back*/); +// read_row(fmemtable, +// rowkey, +// 3000, /*snapshot version*/ +// 1, /*key*/ +// 3 /*value*/); - //release resources - delete[] redo_log_buffer; - encrypt_meta_ = NULL; - lmemtable->destroy(); - fmemtable->destroy(); -} +// //release resources +// delete[] redo_log_buffer; +// encrypt_meta_ = NULL; +// lmemtable->destroy(); +// fmemtable->destroy(); +// } TEST_F(TestMemtableV2, test_compact) { @@ -2388,12 +2444,12 @@ TEST_F(TestMemtableV2, test_compact) lmemtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); write_tx(wtx, lmemtable, 1200, /*snapshot version*/ write_row2); - const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMemtableMutatorIterator mmi; mock_replay_iterator(wtx, mmi); @@ -2410,7 +2466,7 @@ TEST_F(TestMemtableV2, test_compact) lmemtable, 2500, /*snapshot version*/ write_row3); - const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); commit_txn(wtx3, 3000,/*commit_version*/ false/*need_write_back*/); @@ -2621,7 +2677,7 @@ TEST_F(TestMemtableV2, test_compact_v2) memtable, 1000, /*snapshot version*/ rowkey); - const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccTransNode *wtx_first_tnode = get_tx_last_tnode(wtx); ObMvccRow *row = get_tx_last_mvcc_row(wtx); @@ -2629,14 +2685,14 @@ TEST_F(TestMemtableV2, test_compact_v2) memtable, 1200, /*snapshot version*/ write_row); - const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no2 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccTransNode *wtx_second_tnode = get_tx_last_tnode(wtx); write_tx(wtx, memtable, 1300, /*snapshot version*/ write_row2); - const auto wtx_seq_no3 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no3 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccTransNode *wtx_third_tnode = get_tx_last_tnode(wtx); print_callback(wtx); @@ -2651,7 +2707,7 @@ TEST_F(TestMemtableV2, test_compact_v2) memtable, 2500, /*snapshot version*/ write_row3); - const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccTransNode *wtx3_first_tnode = get_tx_last_tnode(wtx3); commit_txn(wtx3, 3000,/*commit_version*/ @@ -2924,7 +2980,7 @@ TEST_F(TestMemtableV2, test_dml_flag) lmemtable, 400, /*snapshot version*/ write_row1); - const auto wtx1_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx1_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccRow *wtx1_row = get_tx_last_mvcc_row(wtx1); ObMvccTransNode *wtx1_tnode1 = get_tx_last_tnode(wtx1); @@ -2955,7 +3011,7 @@ TEST_F(TestMemtableV2, test_dml_flag) lmemtable, 1000, /*snapshot version*/ rowkey); - const auto wtx2_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx2_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMemtableMutatorIterator mmi2; mock_replay_iterator(wtx2, mmi2); @@ -2977,7 +3033,7 @@ TEST_F(TestMemtableV2, test_dml_flag) lmemtable, 1400, /*snapshot version*/ write_row2); - const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx3_seq_no1 = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccTransNode *wtx3_tnode1 = get_tx_last_tnode(wtx3); ObMemtableMutatorIterator mmi3; @@ -3077,7 +3133,7 @@ TEST_F(TestMemtableV2, test_fast_commit) memtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccRowCallback *wtx_cb = (ObMvccRowCallback *)(get_tx_last_cb(wtx)); ObMvccRow *wtx_row = get_tx_last_mvcc_row(wtx); ObMvccTransNode *wtx_tnode = get_tx_last_tnode(wtx); @@ -3197,7 +3253,7 @@ TEST_F(TestMemtableV2, test_fast_commit_with_no_delay_cleanout) memtable, 1000, /*snapshot version*/ write_row); - const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + const auto wtx_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); ObMvccRowCallback *wtx_cb = (ObMvccRowCallback *)(get_tx_last_cb(wtx)); ObMvccRow *wtx_row = get_tx_last_mvcc_row(wtx); ObMvccTransNode *wtx_tnode = get_tx_last_tnode(wtx); @@ -3325,7 +3381,7 @@ TEST_F(TestMemtableV2, test_seq_set_violation) ObTransID write_tx_id = ObTransID(1); ObStoreCtx *wtx = start_tx(write_tx_id); - ObTxSEQ read_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + ObTxSEQ read_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); share::SCN scn_3000; scn_3000.convert_for_tx(3000); start_pdml_stmt(wtx, scn_3000, read_seq_no, 1000000000/*expire_time*/); @@ -3380,7 +3436,7 @@ TEST_F(TestMemtableV2, test_parallel_lock_with_same_txn) // Step1: prepare the global sequence ObSequence::inc(); - ObTxSEQ read_seq_no = ObTxSEQ(ObSequence::get_max_seq_no(),0); + ObTxSEQ read_seq_no = ObTxSEQ(ObSequence::get_max_seq_no()); // Step2: init the mvcc acc ctx wtx->mvcc_acc_ctx_.type_ = ObMvccAccessCtx::T::WRITE; @@ -3389,7 +3445,7 @@ TEST_F(TestMemtableV2, test_parallel_lock_with_same_txn) wtx->mvcc_acc_ctx_.snapshot_.scn_ = read_seq_no; const int64_t abs_expire_time = 10000000000 + ::oceanbase::common::ObTimeUtility::current_time(); wtx->mvcc_acc_ctx_.abs_lock_timeout_ = abs_expire_time; - wtx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no(),0); + wtx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no()); ObTableAccessContext context; ObVersionRange trans_version_range; @@ -3412,13 +3468,84 @@ TEST_F(TestMemtableV2, test_parallel_lock_with_same_txn) rowkey))); // Step4: lock for the second time - wtx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no(),0); + wtx->mvcc_acc_ctx_.tx_scn_ = ObTxSEQ(ObSequence::inc_and_get_max_seq_no()); EXPECT_EQ(OB_SUCCESS, (ret = memtable->lock(iter_param_, context, rowkey))); memtable->destroy(); } +TEST_F(TestMemtableV2, test_sync_log_fail_on_frozen_memtable) +{ + int ret = OB_SUCCESS; + ObMemtable *memtable = create_memtable(); + + TRANS_LOG(INFO, "######## start two txn and write some rows"); + ObDatumRowkey rowkey; + ObTransID txid_1 = ObTransID(1); + ObStoreCtx *tx_1 = start_tx(txid_1); + int i = 1; + for (; i <= 10; i++) { + ObStoreRow row1; + EXPECT_EQ(OB_SUCCESS, mock_row(i, i*2, rowkey, row1)); + write_tx(tx_1, memtable, 10000, row1); + } + ObTransID txid_2 = ObTransID(2); + ObStoreCtx *tx_2 = start_tx(txid_2); + for (; i <= 20; i++) { + ObStoreRow row1; + EXPECT_EQ(OB_SUCCESS, mock_row(i, i*4, rowkey, row1)); + write_tx(tx_2, memtable, 10000, row1); + } + TRANS_LOG(INFO, "######## frozen memtable"); + memtable->set_is_tablet_freeze(); + EXPECT_TRUE(memtable->is_frozen_memtable()); + TRANS_LOG(INFO, "######## submit log for frozen memtable"); + ObCallbackScope scope_tx_1_1; + EXPECT_EQ(OB_SUCCESS, flush_txn_log(tx_1, 100, 5, scope_tx_1_1)); + EXPECT_EQ(5, scope_tx_1_1.cnt_); + ObCallbackScope scope_tx_2_1; + EXPECT_EQ(OB_SUCCESS, flush_txn_log(tx_2, 200, 5, scope_tx_2_1)); + EXPECT_EQ(5, scope_tx_2_1.cnt_); + ObCallbackScope scope_tx_1_2; + EXPECT_EQ(OB_SUCCESS, flush_txn_log(tx_1, 300, 5, scope_tx_1_2)); + EXPECT_EQ(5, scope_tx_1_2.cnt_); + ObCallbackScope scope_tx_2_2; + EXPECT_EQ(OB_SUCCESS, flush_txn_log(tx_2, 400, 5, scope_tx_2_2)); + EXPECT_EQ(5, scope_tx_2_2.cnt_); + EXPECT_EQ(400, memtable->get_max_end_scn().get_val_for_tx()); + EXPECT_EQ(100, memtable->get_rec_scn().get_val_for_tx()); + TRANS_LOG(INFO, "######## sync log fail, adjust memtable's right boundary"); + int64_t removed_cnt_tx_1; + int64_t palf_applied_scn = 250; + EXPECT_EQ(OB_SUCCESS, sync_txn_log_fail(tx_1, scope_tx_1_2, palf_applied_scn, removed_cnt_tx_1)); + EXPECT_EQ(250, memtable->get_max_end_scn().get_val_for_tx()); + EXPECT_EQ(100, memtable->get_rec_scn().get_val_for_tx()); + EXPECT_TRUE(memtable->get_end_scn().is_max()); + int64_t removed_cnt_tx_2; + EXPECT_EQ(OB_SUCCESS, sync_txn_log_fail(tx_2, scope_tx_2_2, palf_applied_scn, removed_cnt_tx_2)); + EXPECT_EQ(removed_cnt_tx_1, 5); + EXPECT_EQ(removed_cnt_tx_2, 5); + EXPECT_EQ(250, memtable->get_max_end_scn().get_val_for_tx()); + EXPECT_EQ(100, memtable->get_rec_scn().get_val_for_tx()); + EXPECT_TRUE(memtable->get_end_scn().is_max()); + TRANS_LOG(INFO, "######## all log sync fail, memtable is empty, adjust memtable's right boundary"); + palf_applied_scn = 50; + EXPECT_EQ(OB_SUCCESS, sync_txn_log_fail(tx_1, scope_tx_1_1, palf_applied_scn, removed_cnt_tx_1)); + EXPECT_EQ(50, memtable->get_max_end_scn().get_val_for_tx()); + EXPECT_EQ(100, memtable->get_rec_scn().get_val_for_tx()); + EXPECT_TRUE(memtable->get_end_scn().is_max()); + EXPECT_EQ(OB_SUCCESS, sync_txn_log_fail(tx_2, scope_tx_2_1, palf_applied_scn, removed_cnt_tx_2)); + EXPECT_EQ(removed_cnt_tx_1, 5); + EXPECT_EQ(removed_cnt_tx_2, 5); + EXPECT_EQ(50, memtable->get_max_end_scn().get_val_for_tx()); + EXPECT_EQ(100, memtable->get_rec_scn().get_val_for_tx()); + EXPECT_TRUE(memtable->get_end_scn().is_max()); + abort_txn(tx_1, true); + abort_txn(tx_2, true); + memtable->destroy(); +} + } // namespace unittest @@ -3471,6 +3598,9 @@ int ObMemtable::lock_row_on_frozen_stores_( return OB_SUCCESS; } } +void ObMemtable::unset_logging_blocked_for_active_memtable() +{ +} } namespace transaction @@ -3515,7 +3645,10 @@ int ObLSTxCtxMgr::init(const int64_t tenant_id, int main(int argc, char **argv) { system("rm -rf test_memtable.log*"); - OB_LOGGER.set_file_name("test_memtable.log"); + OB_LOGGER.set_file_name("test_memtable.log", true, false, + "test_memtable.log", + "test_memtable.log", + "test_memtable.log"); OB_LOGGER.set_log_level("INFO"); STORAGE_LOG(INFO, "begin unittest: test simple memtable"); ::testing::InitGoogleTest(&argc, argv); diff --git a/mittest/mtlenv/storage/test_trans.cpp b/mittest/mtlenv/storage/test_trans.cpp index dc6572a82b..10c79fd073 100644 --- a/mittest/mtlenv/storage/test_trans.cpp +++ b/mittest/mtlenv/storage/test_trans.cpp @@ -455,7 +455,7 @@ int main(int argc, char **argv) { system("rm -rf test_trans.log*"); OB_LOGGER.set_file_name("test_trans.log",true, false, "test_trans.log", "test_trans.log"); - OB_LOGGER.set_log_level("INFO"); + OB_LOGGER.set_log_level("DEBUG"); ::testing::InitGoogleTest(&argc, argv); return RUN_ALL_TESTS(); } diff --git a/mittest/mtlenv/tablelock/table_lock_tx_common_env.h b/mittest/mtlenv/tablelock/table_lock_tx_common_env.h index 26837fb804..dd2bbe5273 100644 --- a/mittest/mtlenv/tablelock/table_lock_tx_common_env.h +++ b/mittest/mtlenv/tablelock/table_lock_tx_common_env.h @@ -32,6 +32,7 @@ public: ObTxData *tx_data, ObTableHandleV2 &lock_memtable_handle); void change_to_leader(); + ~MockObTxCtx(); private: void init_memtable_ctx_(ObTableHandleV2 &lock_memtable_handle); }; @@ -105,6 +106,25 @@ void MockObTxCtx::change_to_leader() role_state_ = TxCtxRoleState::LEADER; } +MockObTxCtx::~MockObTxCtx() +{ + // reset to pass sanity check + mt_ctx_.unsubmitted_cnt_ = 0; + mt_ctx_.log_gen_.reset(); + auto f = [](memtable::ObTxCallbackList &list) -> void { + list.appended_ = 0; + list.synced_ = 0; + list.logged_ = 0; + list.removed_ = 0; + list.length_ = 0; + list.unlog_removed_ = 0; + }; + int c = mt_ctx_.trans_mgr_.get_callback_list_count(); + f(mt_ctx_.trans_mgr_.callback_list_); + for (int i =1; i < c; i++) { + f(mt_ctx_.trans_mgr_.callback_lists_[i - 1]); + } +} void MockObTxCtx::init_memtable_ctx_(ObTableHandleV2 &lock_memtable_handle) { mt_ctx_.is_inited_ = true; @@ -149,7 +169,6 @@ void MockTxEnv::get_store_ctx(MyTxCtx &my_ctx, timeout, timeout, write_flag); - store_ctx.replay_log_scn_ = share::SCN::base_scn(); } } // tablelock diff --git a/mittest/mtlenv/tablelock/test_lock_memtable.cpp b/mittest/mtlenv/tablelock/test_lock_memtable.cpp index 95758bfd93..b22cae65e7 100644 --- a/mittest/mtlenv/tablelock/test_lock_memtable.cpp +++ b/mittest/mtlenv/tablelock/test_lock_memtable.cpp @@ -321,7 +321,9 @@ TEST_F(TestLockMemtable, replay) DEFAULT_IN_TRANS_LOCK_OP.create_schema_version_); ObMemtableMutatorIterator mmi; mmi.table_lock_ = table_lock; + share::SCN log_scn = share::SCN::base_scn(); ret = memtable_.replay_row(store_ctx, + log_scn, &mmi); ASSERT_EQ(OB_SUCCESS, ret); // 1.2 check exist at memctx @@ -740,7 +742,8 @@ TEST_F(TestLockMemtable, out_trans_multi_source) LOG_INFO("TestLockMemtable::out_trans_multi_source 1.1"); is_replay = true; mem_ctx = store_ctx.mvcc_acc_ctx_.mem_ctx_; - ret = mem_ctx->register_multi_source_data_if_need_(lock_op, is_replay); + // should not call register: + // ret = mem_ctx->register_multi_source_data_if_need_(lock_op); ASSERT_EQ(OB_SUCCESS, ret); // 1.2 check exist at memctx LOG_INFO("TestLockMemtable::out_trans_multi_source 1.2"); @@ -760,8 +763,7 @@ TEST_F(TestLockMemtable, out_trans_multi_source) ASSERT_EQ(mds_array.count(), 0); // 2.1 register (not replay) LOG_INFO("TestLockMemtable::out_trans_multi_source 2.1"); - is_replay = false; - ret = mem_ctx->register_multi_source_data_if_need_(lock_op, is_replay); + ret = mem_ctx->register_multi_source_data_if_need_(lock_op); ASSERT_EQ(OB_SUCCESS, ret); // 2.2 check exist at memctx LOG_INFO("TestLockMemtable::out_trans_multi_source 2.2"); @@ -837,9 +839,8 @@ TEST_F(TestLockMemtable, out_trans_multi_source) // 5 NOTIFY UNLOCK // 5.1 register unlock op LOG_INFO("TestLockMemtable::out_trans_multi_source 5.1"); - is_replay = false; mem_ctx = store_ctx2.mvcc_acc_ctx_.mem_ctx_; - ret = mem_ctx->register_multi_source_data_if_need_(unlock_op, is_replay); + ret = mem_ctx->register_multi_source_data_if_need_(unlock_op); ASSERT_EQ(OB_SUCCESS, ret); ret = ctx2.tx_ctx_.gen_total_mds_array_(mds_array_unlock); ASSERT_EQ(OB_SUCCESS, ret); @@ -1009,7 +1010,7 @@ TEST_F(TestLockMemtable, test_lock_retry) start_tx(DEFAULT_TRANS_ID, default_ctx); get_store_ctx(default_ctx, store_ctx); default_ctx.tx_ctx_.change_to_leader(); - default_ctx.tx_ctx_.lock_.lock(); + ASSERT_EQ(OB_SUCCESS, default_ctx.tx_ctx_.lock_.lock()); // 2. do obj lock LOG_INFO("TestLockMemtable::test_lock_retry 2 do obj lock"); diff --git a/mittest/mtlenv/tablelock/test_lock_table_callback.cpp b/mittest/mtlenv/tablelock/test_lock_table_callback.cpp index a53b0ea2b7..7e4ad26834 100644 --- a/mittest/mtlenv/tablelock/test_lock_table_callback.cpp +++ b/mittest/mtlenv/tablelock/test_lock_table_callback.cpp @@ -32,15 +32,6 @@ using namespace share; using namespace storage; using namespace memtable; -namespace memtable -{ - -void ObMemtableCtx::update_max_submitted_seq_no(const ObTxSEQ seq_no) -{ - UNUSEDx(seq_no); -} - -} namespace transaction { namespace tablelock @@ -103,14 +94,11 @@ private: void create_callback(const ObTableLockOp &lock_op, ObOBJLockCallback *&cb) { int ret = OB_SUCCESS; - bool is_replay = false; ObMemCtxLockOpLinkNode *lock_op_node = nullptr; static ObFakeStoreRowKey tablelock_fake_rowkey("tbl", 3); const ObStoreRowkey &rowkey = tablelock_fake_rowkey.get_rowkey(); ObMemtableKey mt_key; - ret = mt_ctx_.lock_mem_ctx_.add_lock_record(lock_op, - lock_op_node, - is_replay); + ret = mt_ctx_.lock_mem_ctx_.add_lock_record(lock_op, lock_op_node); ASSERT_EQ(OB_SUCCESS, ret); cb = mt_ctx_.alloc_table_lock_callback(mt_ctx_, &memtable_); @@ -228,12 +216,13 @@ TEST_F(TestLockTableCallback, basic) create_callback(lock_op, cb); ASSERT_EQ(lock_op.lock_seq_no_, cb->get_seq_no()); ASSERT_EQ(false, cb->must_log()); - ASSERT_EQ(false, cb->log_synced()); + ASSERT_EQ(false, cb->is_log_submitted()); share::SCN scn_10; scn_10.convert_for_logservice(10); - ret = cb->log_sync(scn_10); + ObIMemtable *mt = NULL; + ret = cb->log_submitted_cb(scn_10, mt); ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(true, cb->log_synced()); + ASSERT_EQ(true, cb->is_log_submitted()); ASSERT_EQ(LS_LOCK_TABLET, cb->get_tablet_id()); TableLockRedoDataNode redo_node; diff --git a/mittest/mtlenv/tablelock/test_mem_ctx_table_lock.cpp b/mittest/mtlenv/tablelock/test_mem_ctx_table_lock.cpp index 75fce25305..adac7c948a 100644 --- a/mittest/mtlenv/tablelock/test_mem_ctx_table_lock.cpp +++ b/mittest/mtlenv/tablelock/test_mem_ctx_table_lock.cpp @@ -318,7 +318,8 @@ TEST_F(TestMemCtxTableLock, rollback_table_lock) // 1.3 rollback LOG_INFO("TestMemCtxTableLock::rollback_table_lock 1.3"); auto rollback_to_seq_no = ObTxSEQ(2, 0); - ret = mem_ctx_.rollback_table_lock(rollback_to_seq_no); + auto rollback_from_seq_no = ObTxSEQ(100, 0); + ret = mem_ctx_.rollback_table_lock(rollback_to_seq_no, rollback_from_seq_no); ASSERT_EQ(OB_SUCCESS, ret); ret = mem_ctx_.check_lock_exist(DEFAULT_IN_TRANS_LOCK_OP.lock_id_, DEFAULT_IN_TRANS_LOCK_OP.owner_id_, @@ -339,7 +340,7 @@ TEST_F(TestMemCtxTableLock, rollback_table_lock) // 1.4 rollback again LOG_INFO("TestMemCtxTableLock::rollback_table_lock 1.4"); rollback_to_seq_no = ObTxSEQ(1, 0); - ret = mem_ctx_.rollback_table_lock(rollback_to_seq_no); + ret = mem_ctx_.rollback_table_lock(rollback_to_seq_no, rollback_from_seq_no); ASSERT_EQ(OB_SUCCESS, ret); ret = mem_ctx_.check_lock_exist(DEFAULT_IN_TRANS_LOCK_OP.lock_id_, DEFAULT_IN_TRANS_LOCK_OP.owner_id_, @@ -429,11 +430,12 @@ TEST_F(TestMemCtxTableLock, get_table_lock_store_info) ret = mem_ctx_.get_table_lock_store_info(table_lock_info); ASSERT_EQ(OB_SUCCESS, ret); op_count = table_lock_info.table_lock_ops_.count(); - ASSERT_EQ(0, op_count); + ASSERT_EQ(1, op_count); // 2 CHECK LOGGED LOCK LOG_INFO("TestMemCtxTableLock::check_lock_need_replay 2.1"); scn.set_base(); - mem_ctx_.set_log_synced(lock_op_node, scn); + table_lock_info.reset(); + mem_ctx_.sync_log_succ(scn); ret = mem_ctx_.get_table_lock_store_info(table_lock_info); ASSERT_EQ(OB_SUCCESS, ret); op_count = table_lock_info.table_lock_ops_.count(); @@ -448,13 +450,13 @@ TEST_F(TestMemCtxTableLock, get_table_lock_store_info) ret = mem_ctx_.get_table_lock_store_info(table_lock_info); ASSERT_EQ(OB_SUCCESS, ret); op_count = table_lock_info.table_lock_ops_.count(); - ASSERT_EQ(1, op_count); + ASSERT_EQ(2, op_count); ASSERT_EQ(scn, table_lock_info.max_durable_scn_); // 4 CHECK TWO LOGGED LOG_INFO("TestMemCtxTableLock::check_lock_need_replay 4.1"); table_lock_info.reset(); scn = share::SCN::plus(share::SCN::min_scn(), 2); - mem_ctx_.set_log_synced(lock_op_node, scn); + mem_ctx_.sync_log_succ(scn); ret = mem_ctx_.get_table_lock_store_info(table_lock_info); ASSERT_EQ(OB_SUCCESS, ret); op_count = table_lock_info.table_lock_ops_.count(); diff --git a/mittest/simple_server/env/ob_simple_server_helper.cpp b/mittest/simple_server/env/ob_simple_server_helper.cpp index 8e3f4313d5..934c8dd7b0 100644 --- a/mittest/simple_server/env/ob_simple_server_helper.cpp +++ b/mittest/simple_server/env/ob_simple_server_helper.cpp @@ -712,13 +712,13 @@ int InjectTxFaultHelper::submit_log(const char *buf, const int64_t size, const s { int ret = OB_SUCCESS; - ObTxLogBlockHeader log_block_header; ObSEArray log_list; ObTxLogBlock log_block; int64_t replay_hint = 0; + ObTxLogBlockHeader &log_block_header = log_block.get_header(); if (OB_ISNULL(mgr_)) { ret = OB_ERR_UNEXPECTED; - } else if (OB_FAIL(log_block.init_with_header(buf, size, replay_hint, log_block_header))) { + } else if (OB_FAIL(log_block.init_for_replay(buf, size))) { LOG_WARN("log_block init failed", K(ret), KP(buf), K(size)); } else { while (OB_SUCC(ret)) { diff --git a/mittest/simple_server/test_ob_minor_freeze.cpp b/mittest/simple_server/test_ob_minor_freeze.cpp index dc221ce764..2ac3f873dd 100644 --- a/mittest/simple_server/test_ob_minor_freeze.cpp +++ b/mittest/simple_server/test_ob_minor_freeze.cpp @@ -71,22 +71,6 @@ int64_t ObMemtable::dec_unsubmitted_cnt_() return unsubmitted_cnt; } -int64_t ObMemtable::inc_unsynced_cnt_() -{ - ob_usleep(rand() % SLEEP_TIME); - return ATOMIC_AAF(&unsynced_cnt_, 1); -} - -int64_t ObMemtable::dec_unsynced_cnt_() -{ - ob_usleep(rand() % SLEEP_TIME); - int64_t unsynced_cnt = ATOMIC_SAF(&unsynced_cnt_, 1); - if (unsynced_cnt < 0) { - ob_abort(); - } - return unsynced_cnt; -} - } namespace storage { @@ -105,26 +89,19 @@ namespace checkpoint } // namespace storage namespace transaction { -int ObPartTransCtx::submit_redo_log_for_freeze_(bool &try_submit) +int ObPartTransCtx::submit_redo_log_for_freeze() { int ret = OB_SUCCESS; int64_t sleep_time = rand() % SLEEP_TIME; ob_usleep(sleep_time); - - ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, true); - if (OB_FAIL(check_and_submit_redo_log_(try_submit))) { - TRANS_LOG(WARN, "fail to submit redo log for freeze", K(ret)); - } - if (try_submit && (OB_SUCC(ret) || OB_BLOCK_FROZEN == ret)) { - ret = submit_log_impl_(ObTxLogType::TX_MULTI_DATA_SOURCE_LOG); - } - ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, false); + CtxLockGuard guard(lock_); + bool submitted = false; + ret = submit_redo_log_for_freeze_(submitted); if (sleep_time > 50 && sleep_time < 90) { ret = OB_TX_NOLOGCB; } else if (sleep_time >= 90) { ret = OB_ERR_UNEXPECTED; } - return ret; } } diff --git a/src/logservice/applyservice/ob_log_apply_service.cpp b/src/logservice/applyservice/ob_log_apply_service.cpp index 3714a9ec54..8d5a6a392d 100644 --- a/src/logservice/applyservice/ob_log_apply_service.cpp +++ b/src/logservice/applyservice/ob_log_apply_service.cpp @@ -51,10 +51,11 @@ void ObApplyFsCb::destroy() int ObApplyFsCb::update_end_lsn(int64_t id, const LSN &end_lsn, + const SCN &end_scn, const int64_t proposal_id) { UNUSED(id); - return apply_status_->update_palf_committed_end_lsn(end_lsn, proposal_id); + return apply_status_->update_palf_committed_end_lsn(end_lsn, end_scn, proposal_id); } //---------------ObApplyServiceTask---------------// @@ -256,6 +257,7 @@ ObApplyStatus::ObApplyStatus() proposal_id_(-1), ap_sv_(NULL), palf_committed_end_lsn_(0), + palf_committed_end_scn_(), last_check_scn_(), max_applied_cb_scn_(), submit_task_(), @@ -336,6 +338,7 @@ void ObApplyStatus::destroy() proposal_id_ = -1; role_ = FOLLOWER; fs_cb_.destroy(); + palf_committed_end_scn_.reset(); palf_committed_end_lsn_.reset(); last_check_scn_.reset(); max_applied_cb_scn_.reset(); @@ -609,6 +612,7 @@ int ObApplyStatus::switch_to_follower_() //单线程调用 int ObApplyStatus::update_palf_committed_end_lsn(const palf::LSN &end_lsn, + const SCN &end_scn, const int64_t proposal_id) { int ret = OB_SUCCESS; @@ -626,6 +630,7 @@ int ObApplyStatus::update_palf_committed_end_lsn(const palf::LSN &end_lsn, if (palf_committed_end_lsn_ >= end_lsn) { CLOG_LOG(ERROR, "invalid new end_lsn", KPC(this), K(proposal_id), K(end_lsn)); } else { + palf_committed_end_scn_.atomic_store(end_scn); palf_committed_end_lsn_ = end_lsn; if (OB_FAIL(submit_task_to_apply_service_(submit_task_))) { CLOG_LOG(ERROR, "submit_task_to_apply_service_ failed", KPC(this), K(ret), K(proposal_id), K(end_lsn)); @@ -646,6 +651,12 @@ int ObApplyStatus::update_palf_committed_end_lsn(const palf::LSN &end_lsn, return ret; } +share::SCN ObApplyStatus::get_palf_committed_end_scn() const +{ + share::SCN scn = palf_committed_end_scn_.atomic_load(); + return scn; +} + int ObApplyStatus::unregister_file_size_cb() { int ret = OB_SUCCESS; @@ -1257,6 +1268,26 @@ int ObLogApplyService::get_max_applied_scn(const share::ObLSID &id, SCN &scn) return ret; } +int ObLogApplyService::get_palf_committed_end_scn(const share::ObLSID &id, share::SCN &scn) +{ + int ret = OB_SUCCESS; + ObApplyStatus *apply_status = NULL; + ObApplyStatusGuard guard; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + CLOG_LOG(ERROR, "apply service not init", K(ret)); + } else if (OB_FAIL(get_apply_status(id, guard))) { + CLOG_LOG(WARN, "guard get apply status failed", K(ret), K(id)); + } else if (NULL == (apply_status = guard.get_apply_status())) { + ret = OB_ERR_UNEXPECTED; + CLOG_LOG(WARN, "apply status is not exist", K(ret), K(id)); + } else { + scn = apply_status->get_palf_committed_end_scn(); + CLOG_LOG(TRACE, "apply service get palf_committed_end_lsn success", K(id), K(scn)); + } + return ret; +} + int ObLogApplyService::push_task(ObApplyServiceTask *task) { int ret = OB_SUCCESS; diff --git a/src/logservice/applyservice/ob_log_apply_service.h b/src/logservice/applyservice/ob_log_apply_service.h index 34c2a88a1a..28ff63788f 100644 --- a/src/logservice/applyservice/ob_log_apply_service.h +++ b/src/logservice/applyservice/ob_log_apply_service.h @@ -83,6 +83,7 @@ public: void destroy(); int update_end_lsn(int64_t id, const palf::LSN &end_lsn, + const share::SCN &end_scn, const int64_t proposal_id); private: ObApplyStatus *apply_status_; @@ -173,7 +174,8 @@ public: int switch_to_leader(const int64_t new_proposal_id); int switch_to_follower(); //palf相关 - int update_palf_committed_end_lsn(const palf::LSN &end_lsn, const int64_t proposal_id); + int update_palf_committed_end_lsn(const palf::LSN &end_lsn, const share::SCN &end_scn, const int64_t proposal_id); + share::SCN get_palf_committed_end_scn() const; int unregister_file_size_cb(); void close_palf_handle(); //最大连续回调位点 @@ -209,6 +211,7 @@ public: K(role_), K(proposal_id_), K(palf_committed_end_lsn_), + K(palf_committed_end_scn_), K(last_check_scn_), K(max_applied_cb_scn_)); private: @@ -246,6 +249,7 @@ private: int64_t proposal_id_; ObLogApplyService *ap_sv_; palf::LSN palf_committed_end_lsn_; + share::SCN palf_committed_end_scn_; //LSN standy_committed_end_lsn_; //palf::LSN min_committed_end_lsn_; share::SCN last_check_scn_; //当前待确认的最大连续回调位点 @@ -290,6 +294,7 @@ public: int switch_to_leader(const share::ObLSID &id, const int64_t proposal_id); int switch_to_follower(const share::ObLSID &id); int get_max_applied_scn(const share::ObLSID &id, share::SCN &scn); + int get_palf_committed_end_scn(const share::ObLSID &id, share::SCN &scn); int push_task(ObApplyServiceTask *task); int wait_append_sync(const share::ObLSID &ls_id); int stat_for_each(const common::ObFunction &func); diff --git a/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp b/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp index 5b6576060b..ba68290d2e 100644 --- a/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp +++ b/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp @@ -172,16 +172,17 @@ int ObCDCPartTransResolver::read( int pos = pos_after_log_header; bool is_cluster_id_served = false; transaction::ObTxLogBlock tx_log_block; - transaction::ObTxLogBlockHeader tx_log_block_header; + transaction::ObTxLogBlockHeader *tx_log_block_header = NULL; - if (OB_FAIL(tx_log_block.init(buf, buf_len, pos, tx_log_block_header))) { + if (OB_FAIL(tx_log_block.init_for_replay(buf, buf_len, pos))) { LOG_ERROR("failed to init tx_log_block with header", - KR(ret), K(buf_len), K_(tls_id), K(tx_log_block), K(tx_log_block_header)); - } else if (OB_UNLIKELY(!tx_log_block_header.is_valid())) { + KR(ret), K(buf_len), K_(tls_id), K(tx_log_block), KPC(tx_log_block_header)); + } else if (FALSE_IT(tx_log_block_header = &tx_log_block.get_header())) { + } else if (OB_UNLIKELY(!tx_log_block_header->is_valid())) { ret = OB_ERR_UNEXPECTED; - LOG_ERROR("invalid ObTxLogBlockHeader found in LogEntry", KR(ret), K_(tls_id), K(lsn), K(tx_log_block_header)); - } else if (cluster_id_filter_.check_is_served(tx_log_block_header.get_org_cluster_id(), is_cluster_id_served)) { - LOG_ERROR("check_cluster_id_served failed", KR(ret), K_(tls_id), K(lsn), K(tx_log_block_header)); + LOG_ERROR("invalid ObTxLogBlockHeader found in LogEntry", KR(ret), K_(tls_id), K(lsn), K(pos), KPC(tx_log_block_header), KPHEX(buf, MIN(buf_len, 512))); + } else if (cluster_id_filter_.check_is_served(tx_log_block_header->get_org_cluster_id(), is_cluster_id_served)) { + LOG_ERROR("check_cluster_id_served failed", KR(ret), K_(tls_id), K(lsn), KPC(tx_log_block_header)); } else if (OB_UNLIKELY(!is_cluster_id_served)) { LOG_DEBUG("[STAT] [FETCHER] [TRANS_NOT_SERVE]", K_(tls_id), K(is_cluster_id_served), K(lsn)); } else { @@ -196,7 +197,7 @@ int ObCDCPartTransResolver::read( if (OB_FAIL(read_trans_header_( lsn, - tx_log_block_header.get_tx_id(), + tx_log_block_header->get_tx_id(), missing_info.is_resolving_miss_log(), tx_log_block, tx_header, @@ -204,11 +205,11 @@ int ObCDCPartTransResolver::read( has_redo_in_cur_entry))) { if (OB_ITER_END != ret) { LOG_ERROR("read_trans_header_ from tx_log_block failed", KR(ret), K_(tls_id), K(lsn), - K(tx_log_block_header), K(tx_log_block), K(tx_header), K(has_redo_in_cur_entry), K(tx_log_idx_in_entry)); + KPC(tx_log_block_header), K(tx_log_block), K(tx_header), K(has_redo_in_cur_entry), K(tx_log_idx_in_entry)); } } else if (need_ignore_trans_log_( lsn, - tx_log_block_header.get_tx_id(), + tx_log_block_header->get_tx_id(), tx_header, missing_info, tx_log_idx_in_entry, @@ -217,7 +218,7 @@ int ObCDCPartTransResolver::read( ret = OB_ITER_END; } } else if (OB_FAIL(read_trans_log_( - tx_log_block_header, + *tx_log_block_header, tx_log_block, tx_header, lsn, @@ -226,7 +227,7 @@ int ObCDCPartTransResolver::read( missing_info, has_redo_in_cur_entry))) { if (OB_IN_STOP_STATE != ret) { - LOG_ERROR("read_trans_log_ fail", KR(ret), K_(tls_id), K(tx_log_block_header), + LOG_ERROR("read_trans_log_ fail", KR(ret), K_(tls_id), KPC(tx_log_block_header), K(tx_header), K(has_redo_in_cur_entry)); } } diff --git a/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp b/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp index 436075afac..b49b1379aa 100644 --- a/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp +++ b/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp @@ -2435,11 +2435,24 @@ int PartTransTask::push_rollback_to_info(const palf::LSN &lsn, const ObTxSEQ &ro int ret = OB_SUCCESS; RollbackNode *rollback_node = static_cast(allocator_.alloc(sizeof(RollbackNode))); - if (OB_ISNULL(rollback_node)) { + if (OB_UNLIKELY(! rollback_from.is_valid() || ! rollback_to.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_ERROR("rollback_from_seq and rollback_to_seq should be valid", KR(ret), + K_(tls_id), K_(trans_id), K(lsn), K(rollback_from), K(rollback_to), K_(rollback_list)); + } else if (OB_UNLIKELY(rollback_from.get_branch() != rollback_to.get_branch())) { + ret = OB_INVALID_ARGUMENT; + LOG_ERROR("expect same rollback branch between rollback_from_seq and rollback_to_seq", KR(ret), + K_(tls_id), K_(trans_id), K(lsn), K(rollback_from), K(rollback_to), K_(rollback_list)); + } else if (OB_UNLIKELY(rollback_from <= rollback_to)) { + ret = OB_INVALID_ARGUMENT; + LOG_ERROR("expect rollback_from_seq larger than rollback_to_seq", KR(ret), + K_(tls_id), K_(trans_id), K(lsn), K(rollback_from), K(rollback_to), K_(rollback_list)); + } else if (OB_ISNULL(rollback_node)) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_ERROR("rollback node is null", KR(ret), K_(tls_id), K_(trans_id), K(rollback_from), K(rollback_to)); } else { new(rollback_node) RollbackNode(rollback_from, rollback_to); + if (OB_FAIL(rollback_list_.add(rollback_node))) { LOG_ERROR("rollback_list_ add fail", KR(ret), K_(tls_id), K_(trans_id), K_(rollback_list), KPC(rollback_node)); } else { diff --git a/src/logservice/libobcdc/src/ob_log_rollback_section.cpp b/src/logservice/libobcdc/src/ob_log_rollback_section.cpp index b20ebde119..7a0a3c84a2 100644 --- a/src/logservice/libobcdc/src/ob_log_rollback_section.cpp +++ b/src/logservice/libobcdc/src/ob_log_rollback_section.cpp @@ -10,7 +10,7 @@ * See the Mulan PubL v2 for more details. */ -#define USING_LOG_PREFIX OBLOG +#define USING_LOG_PREFIX OBLOG_PARSER #include "ob_log_rollback_section.h" @@ -39,8 +39,21 @@ bool RollbackNode::is_valid() const bool RollbackNode::should_rollback_stmt(const transaction::ObTxSEQ &stmt_seq_no) const { - // note: from_seq is large than to_seq - return from_seq_ >= stmt_seq_no && to_seq_ < stmt_seq_no; + bool need_rollback = false; + const int64_t rollback_branch_id = from_seq_.get_branch(); + const int64_t stmt_branch_id = stmt_seq_no.get_branch(); + const bool is_branch_rollback = (rollback_branch_id != 0); // branch_id should large than 0 if rollback in branch + const bool need_check_rollback = (! is_branch_rollback) || rollback_branch_id == stmt_branch_id; + + if (need_check_rollback) { + // note: from_seq is large than to_seq + need_rollback = from_seq_ >= stmt_seq_no && to_seq_ < stmt_seq_no; + if (need_rollback) { + LOG_DEBUG("ROLLBACK_STMT", K(is_branch_rollback), K(stmt_seq_no), K_(from_seq), K_(to_seq)); + } + } + + return need_rollback; } } diff --git a/src/logservice/ob_log_handler.cpp b/src/logservice/ob_log_handler.cpp index f84bd1e82c..8d70b3d3c5 100755 --- a/src/logservice/ob_log_handler.cpp +++ b/src/logservice/ob_log_handler.cpp @@ -396,6 +396,22 @@ int ObLogHandler::locate_by_lsn_coarsely(const LSN &lsn, SCN &result_scn) return palf_handle_.locate_by_lsn_coarsely(lsn, result_scn); } +int ObLogHandler::get_max_decided_scn_as_leader(share::SCN &scn) const +{ + int ret = OB_SUCCESS; + share::ObLSID ls_id; + RLockGuard guard(lock_); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + } else if (OB_ISNULL(apply_service_)) { + ret = OB_ERR_UNEXPECTED; + } else if (FALSE_IT(ls_id = id_)) { + } else if (OB_FAIL(apply_service_->get_palf_committed_end_scn(ls_id, scn))) { + CLOG_LOG(WARN, "get palf_committed_end_lsn fail", K(ret), K(id_)); + } + return ret; +} + int ObLogHandler::advance_base_lsn(const LSN &lsn) { RLockGuard guard(lock_); diff --git a/src/logservice/ob_log_handler.h b/src/logservice/ob_log_handler.h index ecdddbf76d..06294f47fe 100755 --- a/src/logservice/ob_log_handler.h +++ b/src/logservice/ob_log_handler.h @@ -101,6 +101,7 @@ public: virtual int locate_by_scn_coarsely(const share::SCN &scn, palf::LSN &result_lsn) = 0; virtual int locate_by_lsn_coarsely(const palf::LSN &lsn, share::SCN &result_scn) = 0; + virtual int get_max_decided_scn_as_leader(share::SCN &scn) const = 0; virtual int advance_base_lsn(const palf::LSN &lsn) = 0; virtual int get_begin_lsn(palf::LSN &lsn) const = 0; virtual int get_end_lsn(palf::LSN &lsn) const = 0; @@ -340,6 +341,19 @@ public: // - OB_NEED_RETRY: the block is being flashback, need retry. // - others: bug int locate_by_lsn_coarsely(const palf::LSN &lsn, share::SCN &result_scn) override final; + // @brief, get max committed scn from applyservice, which is the max scn of log committed by oneself as leader; + // Example: + // At time T1, the replica of the log stream is the leader and the maximum SCN of the logs + // confirmed by this replica is 100. At time T2, the replica switches to being a follower and the + // maximum SCN of the logs synchronized and replayed is 200. At time T3, the log stream replica + // switches back to being the leader and writes logs with SCNs ranging from 201 to 300, all of + // which are not confirmed. In this case, the returned value of the interface would be 100. + // @param[out] max scn of logs confirmed by this replica as being leader + // @return + // OB_NOT_INIT : ls is not inited + // OB_ERR_UNEXPECTED: unexpected error such as apply_service_ is NULL + // OB_SUCCESS + int get_max_decided_scn_as_leader(share::SCN &scn) const override final; // @brief, set the recycable lsn, palf will ensure that the data before recycable lsn readable. // @param[in] const LSN&, recycable lsn. int advance_base_lsn(const palf::LSN &lsn) override final; @@ -662,8 +676,12 @@ public: // @brief, check if replay is enabled. bool is_replay_enabled() const override final; - // @brief, get max decided log ts considering both apply and replay. - // @param[out] int64_t&, max decided log ts ns. + // @brief, get max decided scn considering both apply and replay. + // @param[out] int64_t&, max decided scn. + // @return + // OB_NOT_INIT: not inited + // OB_STATE_NOT_MATCH: ls is offline or stopped + // OB_SUCCESS int get_max_decided_scn(share::SCN &scn) override final; // @brief: store a persistent flag which means this paxos replica can not reply ack when receiving logs. // By default, paxos replica can reply ack. diff --git a/src/logservice/palf/log_sliding_window.cpp b/src/logservice/palf/log_sliding_window.cpp index 5854fef4f7..26de6842e5 100644 --- a/src/logservice/palf/log_sliding_window.cpp +++ b/src/logservice/palf/log_sliding_window.cpp @@ -2170,7 +2170,7 @@ int LogSlidingWindow::sliding_cb(const int64_t sn, const FixedSlidingWindowSlot // Call fs_cb. int tmp_ret = OB_SUCCESS; const int64_t fs_cb_begin_ts = ObTimeUtility::current_time(); - if (OB_SUCCESS != (tmp_ret = palf_fs_cb_->update_end_lsn(palf_id_, log_end_lsn, log_proposal_id))) { + if (OB_SUCCESS != (tmp_ret = palf_fs_cb_->update_end_lsn(palf_id_, log_end_lsn, log_max_scn, log_proposal_id))) { if (OB_EAGAIN == tmp_ret) { if (REACH_TIME_INTERVAL(1 * 1000 * 1000)) { PALF_LOG(WARN, "update_end_lsn eagain", K(tmp_ret), K_(palf_id), K_(self), K(log_id), KPC(log_task)); diff --git a/src/logservice/palf/palf_callback.h b/src/logservice/palf/palf_callback.h index 1d3ef4dfd1..ccb35f8ab4 100644 --- a/src/logservice/palf/palf_callback.h +++ b/src/logservice/palf/palf_callback.h @@ -31,7 +31,7 @@ class PalfFSCb { public: // end_lsn返回的是最后一条已确认日志的下一位置 - virtual int update_end_lsn(int64_t id, const LSN &end_lsn, const int64_t proposal_id) = 0; + virtual int update_end_lsn(int64_t id, const LSN &end_lsn, const share::SCN &end_scn, const int64_t proposal_id) = 0; }; class PalfRoleChangeCb diff --git a/src/logservice/palf/palf_callback_wrapper.cpp b/src/logservice/palf/palf_callback_wrapper.cpp index c5b4cda8a8..be456d698f 100644 --- a/src/logservice/palf/palf_callback_wrapper.cpp +++ b/src/logservice/palf/palf_callback_wrapper.cpp @@ -39,7 +39,7 @@ void PalfFSCbWrapper::del_cb_impl(PalfFSCbNode *cb_impl) (void)list_.remove(cb_impl); } -int PalfFSCbWrapper::update_end_lsn(int64_t id, const LSN &end_lsn, const int64_t proposal_id) +int PalfFSCbWrapper::update_end_lsn(int64_t id, const LSN &end_lsn, const share::SCN &end_scn, const int64_t proposal_id) { int ret = common::OB_SUCCESS; if (OB_UNLIKELY(true == list_.is_empty())) { @@ -52,8 +52,8 @@ int PalfFSCbWrapper::update_end_lsn(int64_t id, const LSN &end_lsn, const int64_ if (NULL == cb) { ret = OB_ERR_UNEXPECTED; PALF_LOG(ERROR, "PalfFSCb is NULL, unexpect error", KPC(node)); - } else if (OB_SUCCESS != (tmp_ret = cb->update_end_lsn(id, end_lsn, proposal_id))) { - PALF_LOG(ERROR, "update_end_lsn failed", K(tmp_ret), K(id), K(end_lsn), K(proposal_id), KPC(node)); + } else if (OB_SUCCESS != (tmp_ret = cb->update_end_lsn(id, end_lsn, end_scn, proposal_id))) { + PALF_LOG(ERROR, "update_end_lsn failed", K(tmp_ret), K(id), K(end_lsn), K(end_scn), K(proposal_id), KPC(node)); } } } diff --git a/src/logservice/palf/palf_callback_wrapper.h b/src/logservice/palf/palf_callback_wrapper.h index 793b81d07f..5b473b4fa7 100644 --- a/src/logservice/palf/palf_callback_wrapper.h +++ b/src/logservice/palf/palf_callback_wrapper.h @@ -38,7 +38,7 @@ public: ~PalfFSCbWrapper(); virtual int add_cb_impl(PalfFSCbNode *cb_impl); virtual void del_cb_impl(PalfFSCbNode *cb_impl); - virtual int update_end_lsn(int64_t id, const LSN &end_lsn, const int64_t proposal_id); + virtual int update_end_lsn(int64_t id, const LSN &end_lsn, const share::SCN &end_scn, const int64_t proposal_id); private: // The head of list ObDList list_; diff --git a/src/logservice/replayservice/ob_replay_status.cpp b/src/logservice/replayservice/ob_replay_status.cpp index ecbfe8873d..f50439820e 100644 --- a/src/logservice/replayservice/ob_replay_status.cpp +++ b/src/logservice/replayservice/ob_replay_status.cpp @@ -533,9 +533,11 @@ int64_t ObLogReplayTask::to_string(char* buf, const int64_t buf_len) const //---------------ObReplayFsCb---------------// int ObReplayFsCb::update_end_lsn(int64_t id, const LSN &end_offset, + const SCN &end_scn, const int64_t proposal_id) { UNUSED(id); + UNUSED(end_scn); UNUSED(proposal_id); return replay_status_->update_end_offset(end_offset); } diff --git a/src/logservice/replayservice/ob_replay_status.h b/src/logservice/replayservice/ob_replay_status.h index b7e1cc9659..65bed115bf 100644 --- a/src/logservice/replayservice/ob_replay_status.h +++ b/src/logservice/replayservice/ob_replay_status.h @@ -391,7 +391,7 @@ public: replay_status_ = NULL; } // 回调接口,调用replay status的update_end_offset接口 - int update_end_lsn(int64_t id, const palf::LSN &end_offset, const int64_t proposal_id); + int update_end_lsn(int64_t id, const palf::LSN &end_offset, const share::SCN &end_scn, const int64_t proposal_id); private: ObReplayStatus *replay_status_; }; diff --git a/src/observer/virtual_table/ob_all_virtual_memstore_info.cpp b/src/observer/virtual_table/ob_all_virtual_memstore_info.cpp index d297e9be2d..1ad38b80ef 100644 --- a/src/observer/virtual_table/ob_all_virtual_memstore_info.cpp +++ b/src/observer/virtual_table/ob_all_virtual_memstore_info.cpp @@ -264,8 +264,9 @@ int ObAllVirtualMemstoreInfo::process_curr_tenant(ObNewRow *&row) cur_row_.cells_[i].set_int(mt->get_unsubmitted_cnt()); break; case OB_APP_MIN_COLUMN_ID + 11: - // unsynced_count - cur_row_.cells_[i].set_int(mt->get_unsynced_cnt()); + // unsynced_count, since 4.3 memtable's unsynced_count is not used + // reuse this field for max_end_scn + cur_row_.cells_[i].set_uint64(mt->get_max_end_scn().get_val_for_inner_table_field()); break; case OB_APP_MIN_COLUMN_ID + 12: // write_ref_count diff --git a/src/observer/virtual_table/ob_all_virtual_minor_freeze_info.cpp b/src/observer/virtual_table/ob_all_virtual_minor_freeze_info.cpp index c3fd88acfc..81ff529737 100644 --- a/src/observer/virtual_table/ob_all_virtual_minor_freeze_info.cpp +++ b/src/observer/virtual_table/ob_all_virtual_minor_freeze_info.cpp @@ -282,8 +282,6 @@ int ObAllVirtualMinorFreezeInfo::generate_memtables_info() append_memtable_info_string(MEMTABLE_INFO_MEMBER[3], to_cstring(memtables_info_[i].write_ref_cnt_), size); // unsubmitted_cnt append_memtable_info_string(MEMTABLE_INFO_MEMBER[4], to_cstring(memtables_info_[i].unsubmitted_cnt_), size); - // unsynced_cnt - append_memtable_info_string(MEMTABLE_INFO_MEMBER[5], to_cstring(memtables_info_[i].unsynced_cnt_), size); // current_right_boundary append_memtable_info_string(MEMTABLE_INFO_MEMBER[6], to_cstring(memtables_info_[i].current_right_boundary_), size); // end of the memtable_info diff --git a/src/observer/virtual_table/ob_all_virtual_tx_stat.cpp b/src/observer/virtual_table/ob_all_virtual_tx_stat.cpp index 00ea187078..e4d20d6ccb 100644 --- a/src/observer/virtual_table/ob_all_virtual_tx_stat.cpp +++ b/src/observer/virtual_table/ob_all_virtual_tx_stat.cpp @@ -299,6 +299,23 @@ int ObGVTxStat::inner_get_next_row(ObNewRow *&row) case TRANSFER_BLOCKING: cur_row_.cells_[i].set_bool(tx_stat.transfer_blocking_); break; + case BUSY_CBS_CNT: + cur_row_.cells_[i].set_int(tx_stat.busy_cbs_cnt_); + break; + case REPLAY_COMPLETE: + cur_row_.cells_[i].set_int(tx_stat.replay_completeness_); + break; + case SERIAL_LOG_FINAL_SCN: + cur_row_.cells_[i].set_int(tx_stat.serial_final_scn_.get_val_for_inner_table_field()); + break; + case CALLBACK_LIST_STATS: + { + const char *buf = to_cstring(tx_stat.get_callback_list_stats_displayer()); + const int32_t buf_len = static_cast(strlen(buf)); + cur_row_.cells_[i].set_lob_value(ObLongTextType, buf, buf_len); + cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + } + break; default: ret = OB_ERR_UNEXPECTED; SERVER_LOG(WARN, "invalid coloum_id", K(ret), K(col_id)); diff --git a/src/observer/virtual_table/ob_all_virtual_tx_stat.h b/src/observer/virtual_table/ob_all_virtual_tx_stat.h index fb6f562454..312099331b 100644 --- a/src/observer/virtual_table/ob_all_virtual_tx_stat.h +++ b/src/observer/virtual_table/ob_all_virtual_tx_stat.h @@ -89,6 +89,10 @@ private: END_SCN, REC_SCN, TRANSFER_BLOCKING, + BUSY_CBS_CNT, + REPLAY_COMPLETE, + SERIAL_LOG_FINAL_SCN, + CALLBACK_LIST_STATS, }; static const int64_t OB_MAX_BUFFER_SIZE = 1024; diff --git a/src/rootserver/ob_recovery_ls_service.cpp b/src/rootserver/ob_recovery_ls_service.cpp index b2b271254f..37fd09f9c7 100755 --- a/src/rootserver/ob_recovery_ls_service.cpp +++ b/src/rootserver/ob_recovery_ls_service.cpp @@ -344,8 +344,7 @@ int ObRecoveryLSService::process_ls_log_( // nothing } else if (logservice::TRANS_SERVICE_LOG_BASE_TYPE == header.get_log_type()) { ObTxLogBlock tx_log_block; - ObTxLogBlockHeader tx_block_header; - if (OB_FAIL(tx_log_block.init(log_buf, log_length, log_pos, tx_block_header))) { + if (OB_FAIL(tx_log_block.init_for_replay(log_buf, log_length, log_pos))) { LOG_WARN("failed to init tx log block", KR(ret), K(log_length)); } else if (OB_FAIL(process_ls_tx_log_(tx_log_block, sync_scn))) { LOG_WARN("failed to process ls tx log", KR(ret), K(tx_log_block), K(sync_scn)); diff --git a/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp b/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp index 1f555698f5..3d60388351 100644 --- a/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp @@ -8028,14 +8028,14 @@ int ObInnerTableSchema::all_virtual_memstore_info_schema(ObTableSchema &table_sc } if (OB_SUCC(ret)) { - ADD_COLUMN_SCHEMA("unsynced_count", //column_name + ADD_COLUMN_SCHEMA("max_end_scn", //column_name ++column_id, //column_id 0, //rowkey_id 0, //index_id 0, //part_key_pos - ObIntType, //column_type + ObUInt64Type, //column_type CS_TYPE_INVALID, //column_collation_type - sizeof(int64_t), //column_length + sizeof(uint64_t), //column_length -1, //column_precision -1, //column_scale false, //is_nullable @@ -8851,6 +8851,66 @@ int ObInnerTableSchema::all_virtual_trans_stat_schema(ObTableSchema &table_schem false, //is_nullable false); //is_autoincrement } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("busy_cbs", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("replay_complete", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("serial_log_final_scn", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("callback_list_stats", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } if (OB_SUCC(ret)) { table_schema.get_part_option().set_part_num(1); table_schema.set_part_level(PARTITION_LEVEL_ONE); diff --git a/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp b/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp index 5c3815af61..8cf28aa3be 100644 --- a/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp @@ -10783,7 +10783,7 @@ int ObInnerTableSchema::all_virtual_memstore_info_ora_schema(ObTableSchema &tabl } if (OB_SUCC(ret)) { - ADD_COLUMN_SCHEMA("UNSYNCED_COUNT", //column_name + ADD_COLUMN_SCHEMA("MAX_END_SCN", //column_name ++column_id, //column_id 0, //rowkey_id 0, //index_id diff --git a/src/share/inner_table/ob_inner_table_schema.15201_15250.cpp b/src/share/inner_table/ob_inner_table_schema.15201_15250.cpp index 06354cbced..95bb2a0637 100644 --- a/src/share/inner_table/ob_inner_table_schema.15201_15250.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15201_15250.cpp @@ -2272,6 +2272,66 @@ int ObInnerTableSchema::all_virtual_trans_stat_ora_schema(ObTableSchema &table_s false, //is_nullable false); //is_autoincrement } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BUSY_CBS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REPLAY_COMPLETE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SERIAL_LOG_FINAL_SCN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("CALLBACK_LIST_STATS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } if (OB_SUCC(ret)) { table_schema.get_part_option().set_part_num(1); table_schema.set_part_level(PARTITION_LEVEL_ONE); diff --git a/src/share/inner_table/ob_inner_table_schema_def.py b/src/share/inner_table/ob_inner_table_schema_def.py index 5036a72695..1b345ccd3b 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -7823,7 +7823,7 @@ def_table_schema( ('logging_blocked', 'varchar:MAX_COLUMN_YES_NO_LENGTH'), ('freeze_clock', 'int'), ('unsubmitted_count', 'int'), - ('unsynced_count', 'int'), + ('max_end_scn', 'uint'), ('write_ref_count', 'int'), ('mem_used', 'int'), ('hash_item_count', 'int'), @@ -7900,6 +7900,10 @@ def_table_schema( ('end_scn', 'uint'), ('rec_scn', 'uint'), ('transfer_blocking', 'bool'), + ('busy_cbs', 'int'), + ('replay_complete', 'int'), + ('serial_log_final_scn', 'int'), + ('callback_list_stats', 'longtext'), ], partition_columns = ['svr_ip', 'svr_port'], vtable_route_policy = 'distributed', diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index 22ec1dfee1..2857b8c263 100644 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -774,7 +774,13 @@ DEF_INT(_tx_result_retention, OB_TENANT_PARAMETER, "300", "[0, 36000]", "The tx data can be recycled after at least _tx_result_retention seconds. " "Range: [0, 36000]", ObParameterAttr(Section::TRANS, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); - +DEF_BOOL(_enable_parallel_redo_logging, OB_CLUSTER_PARAMETER, "True", + "enable parallel write redo log.", + ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); +DEF_CAP(_parallel_redo_logging_trigger, OB_CLUSTER_PARAMETER, "16M", "[0B,)", + "size of single transaction's pending redo log to trigger parallel writes redo log. " + "Range: [0B,+∞)", + ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); DEF_TIME(_ob_get_gts_ahead_interval, OB_CLUSTER_PARAMETER, "0s", "[0s, 1s]", "get gts ahead interval. Range: [0s, 1s]", ObParameterAttr(Section::TRANS, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); @@ -1611,6 +1617,10 @@ DEF_TIME(_balance_wait_killing_transaction_end_threshold, OB_TENANT_PARAMETER, " DEF_BOOL(_enable_px_fast_reclaim, OB_CLUSTER_PARAMETER, "True", "Enable the fast reclaim function through PX tasks deteting for survival by detect manager. The default value is True.", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); +DEF_CAP(_pdml_thread_cache_size, OB_CLUSTER_PARAMETER, "2M", "[1B,)", + "The cache size of per pdml thread." + "Range:[1B, )", + ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); DEF_BOOL(_enable_reserved_user_dcl_restriction, OB_CLUSTER_PARAMETER, "False", "specifies whether to forbid non-reserved user to modify reserved users", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); diff --git a/src/sql/das/ob_das_context.cpp b/src/sql/das/ob_das_context.cpp index 94710d8ea2..2e01a04b5b 100644 --- a/src/sql/das/ob_das_context.cpp +++ b/src/sql/das/ob_das_context.cpp @@ -571,6 +571,7 @@ OB_DEF_SERIALIZE(ObDASCtx) OB_UNIS_ENCODE(flags_); OB_UNIS_ENCODE(snapshot_); OB_UNIS_ENCODE(location_router_); + OB_UNIS_ENCODE(write_branch_id_); return ret; } @@ -601,6 +602,7 @@ OB_DEF_DESERIALIZE(ObDASCtx) LOG_WARN("rebuild tablet loc reference failed", K(ret)); } OB_UNIS_DECODE(location_router_); + OB_UNIS_DECODE(write_branch_id_); return ret; } @@ -615,6 +617,7 @@ OB_DEF_SERIALIZE_SIZE(ObDASCtx) OB_UNIS_ADD_LEN(flags_); OB_UNIS_ADD_LEN(snapshot_); OB_UNIS_ADD_LEN(location_router_); + OB_UNIS_ADD_LEN(write_branch_id_); return len; } } // namespace sql diff --git a/src/sql/das/ob_das_context.h b/src/sql/das/ob_das_context.h index 92205a0b90..6aa852128a 100644 --- a/src/sql/das/ob_das_context.h +++ b/src/sql/das/ob_das_context.h @@ -50,6 +50,7 @@ public: allocator_(allocator), snapshot_(), savepoint_(), + write_branch_id_(0), del_ctx_list_(allocator), jump_read_group_id_(-1), flags_(0) @@ -89,6 +90,8 @@ public: transaction::ObTxReadSnapshot &get_snapshot() { return snapshot_; } transaction::ObTxSEQ get_savepoint() const { return savepoint_; } void set_savepoint(const transaction::ObTxSEQ savepoint) { savepoint_ = savepoint; } + void set_write_branch_id(const int16_t branch_id) { write_branch_id_ = branch_id; } + int16_t get_write_branch_id() const { return write_branch_id_; } ObDASLocationRouter &get_location_router() { return location_router_; } int build_related_tablet_loc(ObDASTabletLoc &tablet_loc); int build_related_table_loc(ObDASTableLoc &table_loc); @@ -114,7 +117,8 @@ public: K_(external_table_locs), K_(is_fk_cascading), K_(snapshot), - K_(savepoint)); + K_(savepoint), + K_(write_branch_id)); private: int check_same_server(const ObDASTabletLoc *tablet_loc); private: @@ -131,6 +135,11 @@ private: common::ObIAllocator &allocator_; transaction::ObTxReadSnapshot snapshot_; // Mvcc snapshot transaction::ObTxSEQ savepoint_; // DML savepoint + // for DML like `insert update` and `replace`, which use savepoint to + // resolve conflicts and when these DML executed under partition-wise + // style, they need rollback their own writes but not all, we assign + // id to data writes by different writer thread (named branch) + int16_t write_branch_id_; //@todo: save snapshot version DASDelCtxList del_ctx_list_; public: diff --git a/src/sql/das/ob_das_delete_op.cpp b/src/sql/das/ob_das_delete_op.cpp index 2e0e2a3d5b..7e9f94e1a5 100644 --- a/src/sql/das/ob_das_delete_op.cpp +++ b/src/sql/das/ob_das_delete_op.cpp @@ -98,6 +98,7 @@ int ObDASDeleteOp::open_op() ObDASIndexDMLAdaptor del_adaptor; del_adaptor.tx_desc_ = trans_desc_; del_adaptor.snapshot_ = snapshot_; + del_adaptor.write_branch_id_ = write_branch_id_; del_adaptor.ctdef_ = del_ctdef_; del_adaptor.rtdef_ = del_rtdef_; del_adaptor.related_ctdefs_ = &related_ctdefs_; diff --git a/src/sql/das/ob_das_insert_op.cpp b/src/sql/das/ob_das_insert_op.cpp index c41ee91303..2b98004207 100644 --- a/src/sql/das/ob_das_insert_op.cpp +++ b/src/sql/das/ob_das_insert_op.cpp @@ -127,6 +127,7 @@ int ObDASInsertOp::insert_rows() ObDASIndexDMLAdaptor ins_adaptor; ins_adaptor.tx_desc_ = trans_desc_; ins_adaptor.snapshot_ = snapshot_; + ins_adaptor.write_branch_id_ = write_branch_id_; ins_adaptor.ctdef_ = ins_ctdef_; ins_adaptor.rtdef_ = ins_rtdef_; ins_adaptor.related_ctdefs_ = &related_ctdefs_; @@ -159,7 +160,7 @@ int ObDASInsertOp::insert_row_with_fetch() if (ins_ctdef_->table_rowkey_types_.empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_rowkey_types is invalid", K(ret)); - } else if (OB_FAIL(ObDMLService::init_dml_param(*ins_ctdef_, *ins_rtdef_, *snapshot_, op_alloc_, dml_param))) { + } else if (OB_FAIL(ObDMLService::init_dml_param(*ins_ctdef_, *ins_rtdef_, *snapshot_, write_branch_id_, op_alloc_, dml_param))) { LOG_WARN("init dml param failed", K(ret), KPC_(ins_ctdef), KPC_(ins_rtdef)); } else if (OB_ISNULL(buf = op_alloc_.alloc(sizeof(ObDASConflictIterator)))) { ret = OB_ERR_UNEXPECTED; @@ -210,6 +211,7 @@ int ObDASInsertOp::insert_row_with_fetch() } else if (OB_FAIL(ObDMLService::init_dml_param(*index_ins_ctdef, *index_ins_rtdef, *snapshot_, + write_branch_id_, op_alloc_, dml_param))) { LOG_WARN("init index dml param failed", K(ret), KPC(index_ins_ctdef), KPC(index_ins_rtdef)); diff --git a/src/sql/das/ob_das_lock_op.cpp b/src/sql/das/ob_das_lock_op.cpp index 9f03a38207..1721304349 100644 --- a/src/sql/das/ob_das_lock_op.cpp +++ b/src/sql/das/ob_das_lock_op.cpp @@ -55,7 +55,7 @@ int ObDASLockOp::open_op() ObDASDMLIterator dml_iter(lock_ctdef_, lock_buffer_, op_alloc_); ObAccessService *as = MTL(ObAccessService *); - if (OB_FAIL(ObDMLService::init_dml_param(*lock_ctdef_, *lock_rtdef_, *snapshot_, op_alloc_, dml_param))) { + if (OB_FAIL(ObDMLService::init_dml_param(*lock_ctdef_, *lock_rtdef_, *snapshot_, write_branch_id_, op_alloc_, dml_param))) { LOG_WARN("init dml param failed", K(ret)); } else if (OB_FAIL(as->lock_rows(ls_id_, tablet_id_, diff --git a/src/sql/das/ob_das_ref.cpp b/src/sql/das/ob_das_ref.cpp index 99c9c92c18..654e672cf8 100644 --- a/src/sql/das/ob_das_ref.cpp +++ b/src/sql/das/ob_das_ref.cpp @@ -515,6 +515,7 @@ int ObDASRef::create_das_task(const ObDASTabletLoc *tablet_loc, } else { task_op->set_trans_desc(session->get_tx_desc()); task_op->set_snapshot(&get_exec_ctx().get_das_ctx().get_snapshot()); + task_op->set_write_branch_id(get_exec_ctx().get_das_ctx().get_write_branch_id()); task_op->set_tenant_id(session->get_effective_tenant_id()); task_op->set_task_id(task_id); task_op->in_stmt_retry_ = session->get_is_in_retry(); diff --git a/src/sql/das/ob_das_task.h b/src/sql/das/ob_das_task.h index d42e8f87b6..3f0b4ce8b8 100644 --- a/src/sql/das/ob_das_task.h +++ b/src/sql/das/ob_das_task.h @@ -111,6 +111,7 @@ public: task_flag_(0), trans_desc_(nullptr), snapshot_(nullptr), + write_branch_id_(0), tablet_loc_(nullptr), op_alloc_(op_alloc), related_ctdefs_(op_alloc), @@ -192,6 +193,8 @@ public: transaction::ObTxDesc *get_trans_desc() { return trans_desc_; } void set_snapshot(transaction::ObTxReadSnapshot *snapshot) { snapshot_ = snapshot; } transaction::ObTxReadSnapshot *get_snapshot() { return snapshot_; } + int16_t get_write_branch_id() const { return write_branch_id_; } + void set_write_branch_id(const int16_t branch_id) { write_branch_id_ = branch_id; } bool is_local_task() const { return task_started_; } void set_can_part_retry(const bool flag) { can_part_retry_ = flag; } bool can_part_retry() const { return can_part_retry_; } @@ -245,6 +248,7 @@ protected: }; transaction::ObTxDesc *trans_desc_; //trans desc,事务是全局信息,由RPC框架管理,这里不维护其内存 transaction::ObTxReadSnapshot *snapshot_; // Mvcc snapshot + int16_t write_branch_id_; // branch id for parallel write, required for partially rollback common::ObTabletID tablet_id_; share::ObLSID ls_id_; const ObDASTabletLoc *tablet_loc_; //does not need serialize it diff --git a/src/sql/das/ob_das_update_op.cpp b/src/sql/das/ob_das_update_op.cpp index 741da4720e..1a75db6e57 100644 --- a/src/sql/das/ob_das_update_op.cpp +++ b/src/sql/das/ob_das_update_op.cpp @@ -318,6 +318,7 @@ int ObDASUpdateOp::open_op() ObDASIndexDMLAdaptor upd_adaptor; upd_adaptor.tx_desc_ = trans_desc_; upd_adaptor.snapshot_ = snapshot_; + upd_adaptor.write_branch_id_ = write_branch_id_; upd_adaptor.ctdef_ = upd_ctdef_; upd_adaptor.rtdef_ = upd_rtdef_; upd_adaptor.related_ctdefs_ = &related_ctdefs_; diff --git a/src/sql/engine/dml/ob_dml_service.cpp b/src/sql/engine/dml/ob_dml_service.cpp index 30e4738215..d1f886f2c4 100644 --- a/src/sql/engine/dml/ob_dml_service.cpp +++ b/src/sql/engine/dml/ob_dml_service.cpp @@ -1141,6 +1141,7 @@ int ObDMLService::delete_row(const ObDASDelCtDef &das_del_ctdef, int ObDMLService::init_dml_param(const ObDASDMLBaseCtDef &base_ctdef, ObDASDMLBaseRtDef &base_rtdef, transaction::ObTxReadSnapshot &snapshot, + const int16_t write_branch_id, ObIAllocator &das_alloc, storage::ObDMLBaseParam &dml_param) { @@ -1157,6 +1158,7 @@ int ObDMLService::init_dml_param(const ObDASDMLBaseCtDef &base_ctdef, dml_param.is_batch_stmt_ = base_ctdef.is_batch_stmt_; dml_param.dml_allocator_ = &das_alloc; dml_param.snapshot_ = snapshot; + dml_param.branch_id_ = write_branch_id; if (base_ctdef.is_batch_stmt_) { dml_param.write_flag_.set_is_dml_batch_opt(); } diff --git a/src/sql/engine/dml/ob_dml_service.h b/src/sql/engine/dml/ob_dml_service.h index d048776ee9..fa2805debe 100644 --- a/src/sql/engine/dml/ob_dml_service.h +++ b/src/sql/engine/dml/ob_dml_service.h @@ -140,6 +140,7 @@ public: static int init_dml_param(const ObDASDMLBaseCtDef &base_ctdef, ObDASDMLBaseRtDef &base_rtdef, transaction::ObTxReadSnapshot &snapshot, + const int16_t write_branch_id, common::ObIAllocator &das_alloc, storage::ObDMLBaseParam &dml_param); static int init_das_dml_rtdef(ObDMLRtCtx &dml_rtctx, @@ -281,6 +282,7 @@ public: ObDASIndexDMLAdaptor() : tx_desc_(nullptr), snapshot_(nullptr), + write_branch_id_(0), ctdef_(nullptr), rtdef_(nullptr), related_ctdefs_(nullptr), @@ -302,6 +304,7 @@ public: public: transaction::ObTxDesc *tx_desc_; transaction::ObTxReadSnapshot *snapshot_; + int16_t write_branch_id_; const CtDefType *ctdef_; RtDefType *rtdef_; const DASCtDefFixedArray *related_ctdefs_; @@ -325,7 +328,7 @@ int ObDASIndexDMLAdaptor::write_tablet(DMLIterator &iter, int64_ if (OB_FAIL(write_tablet_with_ignore(iter, affected_rows))) { LOG_WARN("write tablet with ignore failed", K(ret)); } - } else if (OB_FAIL(ObDMLService::init_dml_param(*ctdef_, *rtdef_, *snapshot_, *das_allocator_, dml_param_))) { + } else if (OB_FAIL(ObDMLService::init_dml_param(*ctdef_, *rtdef_, *snapshot_, write_branch_id_, *das_allocator_, dml_param_))) { SQL_DAS_LOG(WARN, "init dml param failed", K(ret), K(ctdef_->table_id_), K(ctdef_->index_tid_)); } else if (OB_FAIL(write_rows(ls_id_, tablet_id_, *ctdef_, *rtdef_, iter, affected_rows))) { SQL_DAS_LOG(WARN, "write rows failed", K(ret), @@ -341,7 +344,7 @@ int ObDASIndexDMLAdaptor::write_tablet(DMLIterator &iter, int64_ K(ls_id_), K(related_tablet_id), K(related_ctdef->table_id_), K(related_ctdef->index_tid_)); if (OB_FAIL(iter.rewind(related_ctdef))) { SQL_DAS_LOG(WARN, "rewind iterator failed", K(ret)); - } else if (OB_FAIL(ObDMLService::init_dml_param(*related_ctdef, *related_rtdef, *snapshot_, *das_allocator_, dml_param_))) { + } else if (OB_FAIL(ObDMLService::init_dml_param(*related_ctdef, *related_rtdef, *snapshot_, write_branch_id_, *das_allocator_, dml_param_))) { SQL_DAS_LOG(WARN, "init index dml param failed", K(ret), K(related_ctdef->table_id_), K(related_ctdef->index_tid_)); } else if (OB_FAIL(write_rows(ls_id_, @@ -398,7 +401,7 @@ int ObDASIndexDMLAdaptor::write_tablet_with_ignore(DMLIterator & SQL_DAS_LOG(TRACE, "write table dml row with ignore", KPC(dml_row), K(ls_id_), K(tablet_id_), K(ctdef_->table_id_), K(ctdef_->index_tid_)); DMLIterator single_row_iter(ctdef_, single_row_buffer, *das_allocator_); - if (OB_FAIL(ObDMLService::init_dml_param(*ctdef_, *rtdef_, *snapshot_, *das_allocator_, dml_param_))) { + if (OB_FAIL(ObDMLService::init_dml_param(*ctdef_, *rtdef_, *snapshot_, write_branch_id_, *das_allocator_, dml_param_))) { SQL_DAS_LOG(WARN, "init dml param failed", K(ret), KPC_(ctdef), KPC_(rtdef)); } else if (OB_FAIL(write_rows(ls_id_, tablet_id_, @@ -422,6 +425,7 @@ int ObDASIndexDMLAdaptor::write_tablet_with_ignore(DMLIterator & } else if (OB_FAIL(ObDMLService::init_dml_param(*related_ctdef, *related_rtdef, *snapshot_, + write_branch_id_, *das_allocator_, dml_param_))) { SQL_DAS_LOG(WARN, "init index dml param failed", K(ret), diff --git a/src/sql/engine/dml/ob_table_modify_op.cpp b/src/sql/engine/dml/ob_table_modify_op.cpp index 4cad7ab936..9a939d98cf 100644 --- a/src/sql/engine/dml/ob_table_modify_op.cpp +++ b/src/sql/engine/dml/ob_table_modify_op.cpp @@ -747,6 +747,7 @@ int ObTableModifyOp::inner_open() } else { init_das_dml_ctx(); } + LOG_TRACE("table_modify_op", K(execute_single_row_)); return ret; } diff --git a/src/sql/engine/ob_exec_context.h b/src/sql/engine/ob_exec_context.h index 82133b78de..cc24b6aa47 100644 --- a/src/sql/engine/ob_exec_context.h +++ b/src/sql/engine/ob_exec_context.h @@ -291,7 +291,8 @@ public: bool has_non_trivial_expr_op_ctx() const { return has_non_trivial_expr_op_ctx_; } void set_non_trivial_expr_op_ctx(bool v) { has_non_trivial_expr_op_ctx_ = v; } inline bool &get_tmp_alloc_used() { return tmp_alloc_used_; } - + // set write branch id for DML write + void set_branch_id(const int16_t branch_id) { das_ctx_.set_write_branch_id(branch_id); } VIRTUAL_NEED_SERIALIZE_AND_DESERIALIZE; protected: uint64_t get_ser_version() const; diff --git a/src/sql/engine/pdml/static/ob_pdml_op_batch_row_cache.cpp b/src/sql/engine/pdml/static/ob_pdml_op_batch_row_cache.cpp index 99adcb7227..979c630554 100644 --- a/src/sql/engine/pdml/static/ob_pdml_op_batch_row_cache.cpp +++ b/src/sql/engine/pdml/static/ob_pdml_op_batch_row_cache.cpp @@ -180,7 +180,7 @@ int ObPDMLOpBatchRowCache::add_row(const ObExprPtrIArray &row, ObTabletID tablet // the 2MB config is tested optimal under PDML concurrency=4 and concurrency=8 cases // TODO: maybe we can introduce a dynamic control policy // concidering the tenant overall access behavior to storage - constexpr int64_t max_pdml_cache_size_per_thread = 2 * 1024 * 1024; + const int64_t max_pdml_cache_size_per_thread = GCONF._pdml_thread_cache_size; if (!with_barrier_ && cached_rows_size_ > max_pdml_cache_size_per_thread) { ret = OB_EXCEED_MEM_LIMIT; } else if (OB_FAIL(process_dump())) { diff --git a/src/sql/engine/px/ob_dfo.cpp b/src/sql/engine/px/ob_dfo.cpp index 6f13c09481..88df2fe1e9 100644 --- a/src/sql/engine/px/ob_dfo.cpp +++ b/src/sql/engine/px/ob_dfo.cpp @@ -66,7 +66,8 @@ OB_SERIALIZE_MEMBER(ObPxSqcMeta, px_detectable_ids_, p2p_dh_map_info_, sqc_count_, - monitoring_info_); + monitoring_info_, + branch_id_base_); OB_SERIALIZE_MEMBER(ObPxTask, qc_id_, dfo_id_, @@ -79,7 +80,8 @@ OB_SERIALIZE_MEMBER(ObPxTask, exec_addr_, execution_id_, px_int_id_, - is_fulltree_); + is_fulltree_, + branch_id_); OB_SERIALIZE_MEMBER(ObPxRpcInitTaskResponse, task_co_id_); @@ -155,6 +157,7 @@ int ObPxSqcMeta::assign(const ObPxSqcMeta &other) qc_id_ = other.qc_id_; dfo_id_ = other.dfo_id_; sqc_id_ = other.sqc_id_; + branch_id_base_ = other.branch_id_base_; thread_inited_ = other.thread_inited_; thread_finish_ = other.thread_finish_; exec_addr_ = other.exec_addr_; @@ -1000,11 +1003,7 @@ int ObPxRpcInitTaskArgs::deep_copy_assign(ObPxRpcInitTaskArgs &src, } else if (ser_pos != des_pos) { ret = OB_DESERIALIZE_ERROR; LOG_WARN("data_len and pos mismatch", K(ser_arg_len), K(ser_pos), K(des_pos), K(ret)); - } else { - // PLACE_HOLDER: if want multiple px worker share trans_desc - // set exec_ctx_->session->set_effective_trans_desc(src.exec_ctx_->session->get_effective_trans_desc()); } - return ret; } diff --git a/src/sql/engine/px/ob_dfo.h b/src/sql/engine/px/ob_dfo.h index 29f01ef412..8ed35fb08d 100644 --- a/src/sql/engine/px/ob_dfo.h +++ b/src/sql/engine/px/ob_dfo.h @@ -207,6 +207,7 @@ public: qc_id_(common::OB_INVALID_ID), sqc_id_(common::OB_INVALID_ID), dfo_id_(common::OB_INVALID_ID), + branch_id_base_(0), access_table_locations_(), qc_ch_info_(), sqc_ch_info_(), @@ -368,8 +369,10 @@ public: bool sqc_order_gi_tasks() const { return sqc_order_gi_tasks_; } ObQCMonitoringInfo &get_monitoring_info() { return monitoring_info_; } const ObQCMonitoringInfo &get_monitoring_info() const { return monitoring_info_; } + void set_branch_id_base(const int16_t branch_id_base) { branch_id_base_ = branch_id_base; } + int16_t get_branch_id_base() const { return branch_id_base_; } TO_STRING_KV(K_(need_report), K_(execution_id), K_(qc_id), K_(sqc_id), K_(dfo_id), K_(exec_addr), K_(qc_addr), - K_(qc_ch_info), K_(sqc_ch_info), + K_(branch_id_base), K_(qc_ch_info), K_(sqc_ch_info), K_(task_count), K_(max_task_count), K_(min_task_count), K_(thread_inited), K_(thread_finish), K_(px_int_id), K_(transmit_use_interm_result), @@ -380,6 +383,9 @@ private: uint64_t qc_id_; int64_t sqc_id_; int64_t dfo_id_; + // branch id is used to distinguish datas written concurrently by px-workers + // for replace and insert update operator, they need branch_id to rollback writes by one px-worker + int16_t branch_id_base_; ObQCMonitoringInfo monitoring_info_; // The partition location information of the all table_scan op and dml op // used for px worker execution @@ -472,6 +478,7 @@ public: child_dfos_(), has_scan_(false), has_dml_op_(false), + has_need_branch_id_op_(false), has_temp_scan_(false), is_active_(false), is_scheduled_(false), @@ -541,6 +548,8 @@ public: inline bool has_scan_op() const { return has_scan_; } inline void set_dml_op(bool has_dml_op) { has_dml_op_ = has_dml_op; } inline bool has_dml_op() { return has_dml_op_; } + inline void set_need_branch_id_op(bool has_need_branch_id_op) { has_need_branch_id_op_ = has_need_branch_id_op; } + inline bool has_need_branch_id_op() const { return has_need_branch_id_op_; } inline void set_temp_table_scan(bool has_scan) { has_temp_scan_ = has_scan; } inline bool has_temp_table_scan() const { return has_temp_scan_; } inline bool is_fast_dfo() const { return is_prealloc_receive_channel() || is_prealloc_transmit_channel(); } @@ -741,6 +750,7 @@ private: common::ObSEArray child_dfos_; bool has_scan_; // DFO 中包含至少一个 scan 算子,或者仅仅包含一个dml bool has_dml_op_; // DFO中可能包含一个dml + bool has_need_branch_id_op_; // DFO 中有算子需要分配branch_id bool has_temp_scan_; bool is_active_; bool is_scheduled_; @@ -906,6 +916,7 @@ public: dfo_id_(0), sqc_id_(0), task_id_(-1), + branch_id_(0), execution_id_(0), task_channel_(NULL), sqc_channel_(NULL), @@ -933,6 +944,7 @@ public: dfo_id_ = other.dfo_id_; sqc_id_ = other.sqc_id_; task_id_ = other.task_id_; + branch_id_ = other.branch_id_; execution_id_ = other.execution_id_; sqc_ch_info_ = other.sqc_ch_info_; task_ch_info_ = other.task_ch_info_; @@ -958,6 +970,7 @@ public: K_(dfo_id), K_(sqc_id), K_(task_id), + K_(branch_id), K_(execution_id), K_(sqc_ch_info), K_(task_ch_info), @@ -986,6 +999,8 @@ public: inline bool is_task_state_set(int32_t flag) const { return 0 != (state_ & flag); } inline void set_task_id(int64_t task_id) { task_id_ = task_id; } inline int64_t get_task_id() const { return task_id_; } + inline void set_branch_id(int16_t branch_id) { branch_id_ = branch_id; } + inline int16_t get_branch_id() const { return branch_id_; } inline void set_qc_id(uint64_t qc_id) { qc_id_ = qc_id; } inline int64_t get_qc_id() const { return qc_id_; } inline void set_sqc_id(int64_t sqc_id) { sqc_id_ = sqc_id; } @@ -1029,6 +1044,7 @@ public: int64_t dfo_id_; int64_t sqc_id_; int64_t task_id_; + int16_t branch_id_; int64_t execution_id_; dtl::ObDtlChannelInfo sqc_ch_info_; dtl::ObDtlChannelInfo task_ch_info_; diff --git a/src/sql/engine/px/ob_dfo_mgr.cpp b/src/sql/engine/px/ob_dfo_mgr.cpp index b3962a3365..7bbb9e1bd0 100644 --- a/src/sql/engine/px/ob_dfo_mgr.cpp +++ b/src/sql/engine/px/ob_dfo_mgr.cpp @@ -502,6 +502,11 @@ int ObDfoMgr::do_split(ObExecContext &exec_ctx, } else if (phy_op->is_dml_operator() && NULL != parent_dfo) { // 当前op是一个dml算子,需要设置dfo的属性 parent_dfo->set_dml_op(true); + const ObPhyOperatorType op_type = phy_op->get_type(); + LOG_TRACE("set DFO need_branch_id", K(op_type)); + parent_dfo->set_need_branch_id_op(op_type == PHY_INSERT_ON_DUP + || op_type == PHY_REPLACE + || op_type == PHY_LOCK); } else if (phy_op->get_type() == PHY_TEMP_TABLE_ACCESS && NULL != parent_dfo) { parent_dfo->set_temp_table_scan(true); const ObTempTableAccessOpSpec *access = static_cast(phy_op); diff --git a/src/sql/engine/px/ob_dfo_scheduler.cpp b/src/sql/engine/px/ob_dfo_scheduler.cpp index 53455ecb80..f86455436b 100644 --- a/src/sql/engine/px/ob_dfo_scheduler.cpp +++ b/src/sql/engine/px/ob_dfo_scheduler.cpp @@ -584,6 +584,19 @@ int ObSerialDfoScheduler::do_schedule_dfo(ObExecContext &ctx, ObDfo &dfo) const } } + // 2. allocate branch_id for DML: replace, insert update, select for update + if (OB_SUCC(ret) && dfo.has_need_branch_id_op()) { + ARRAY_FOREACH_X(sqcs, idx, cnt, OB_SUCC(ret)) { + int16_t branch_id = 0; + const int64_t max_task_count = sqcs.at(idx)->get_max_task_count(); + if (OB_FAIL(ObSqlTransControl::alloc_branch_id(ctx, max_task_count, branch_id))) { + LOG_WARN("alloc branch id fail", KR(ret), K(max_task_count)); + } else { + sqcs.at(idx)->set_branch_id_base(branch_id); + LOG_TRACE("alloc branch id", K(max_task_count), K(branch_id), KPC(sqcs.at(idx))); + } + } + } if (OB_SUCC(ret)) { if (OB_FAIL(dispatch_sqcs(ctx, dfo, sqcs))) { @@ -755,6 +768,20 @@ int ObParallelDfoScheduler::do_schedule_dfo(ObExecContext &exec_ctx, ObDfo &dfo) } } + // 2. allocate branch_id for DML: replace, insert update, select for update + if (OB_SUCC(ret) && dfo.has_need_branch_id_op()) { + ARRAY_FOREACH_X(sqcs, idx, cnt, OB_SUCC(ret)) { + int16_t branch_id = 0; + const int64_t max_task_count = sqcs.at(idx)->get_max_task_count(); + if (OB_FAIL(ObSqlTransControl::alloc_branch_id(exec_ctx, max_task_count, branch_id))) { + LOG_WARN("alloc branch id fail", KR(ret), K(max_task_count)); + } else { + sqcs.at(idx)->set_branch_id_base(branch_id); + LOG_TRACE("alloc branch id", K(max_task_count), K(branch_id), KPC(sqcs.at(idx))); + } + } + } + if (OB_SUCC(ret)) { // 下面的逻辑处理握手阶段超时的情况 // - 目的: 为了防止死锁 diff --git a/src/sql/engine/px/ob_px_sub_coord.cpp b/src/sql/engine/px/ob_px_sub_coord.cpp index 2321b7c845..98ff0d1aa0 100644 --- a/src/sql/engine/px/ob_px_sub_coord.cpp +++ b/src/sql/engine/px/ob_px_sub_coord.cpp @@ -526,6 +526,9 @@ int ObPxSubCoord::create_tasks(ObPxRpcInitSqcArgs &sqc_arg, ObSqcCtx &sqc_ctx, b const ObAddr &task_exec_addr = sqc.get_exec_addr(); const ObAddr &qc_exec_addr = sqc.get_qc_addr(); task.set_task_id(i); + if (sqc.get_branch_id_base()) { + task.set_branch_id(sqc.get_branch_id_base() + i); + } task.set_sqc_addr(sqc_exec_addr); task.set_exec_addr(task_exec_addr); task.set_qc_addr(qc_exec_addr); diff --git a/src/sql/engine/px/ob_px_task_process.cpp b/src/sql/engine/px/ob_px_task_process.cpp index 35c03cd430..d9e4d089ee 100644 --- a/src/sql/engine/px/ob_px_task_process.cpp +++ b/src/sql/engine/px/ob_px_task_process.cpp @@ -163,6 +163,7 @@ int ObPxTaskProcess::process() arg_.exec_ctx_->set_sqc_handler(arg_.sqc_handler_); arg_.exec_ctx_->set_px_task_id(arg_.task_.get_task_id()); arg_.exec_ctx_->set_px_sqc_id(arg_.task_.get_sqc_id()); + arg_.exec_ctx_->set_branch_id(arg_.task_.get_branch_id()); ObMaxWaitGuard max_wait_guard(enable_perf_event ? &max_wait_desc : NULL); ObTotalWaitGuard total_wait_guard(enable_perf_event ? &total_wait_desc : NULL); diff --git a/src/sql/ob_sql_trans_control.cpp b/src/sql/ob_sql_trans_control.cpp index 25c2ff453e..01cc24d7c4 100644 --- a/src/sql/ob_sql_trans_control.cpp +++ b/src/sql/ob_sql_trans_control.cpp @@ -1136,9 +1136,10 @@ int ObSqlTransControl::create_anonymous_savepoint(ObExecContext &exec_ctx, trans ObSQLSessionInfo *session = GET_MY_SESSION(exec_ctx); CK (OB_NOT_NULL(session)); OZ (get_tx_service(session, txs)); + CK (OB_NOT_NULL(session->get_tx_desc())); ObTxParam tx_param; - OZ (build_tx_param_(session, tx_param)); - OZ (txs->create_implicit_savepoint(*session->get_tx_desc(), tx_param, savepoint), *session->get_tx_desc()); + const int16_t branch_id = DAS_CTX(exec_ctx).get_write_branch_id(); + OZ (txs->create_branch_savepoint(*session->get_tx_desc(), branch_id, savepoint), *session->get_tx_desc()); return ret; } @@ -1496,5 +1497,16 @@ int ObSqlTransControl::check_free_route_tx_alive(ObSQLSessionInfo &session, tran return ret; } +int ObSqlTransControl::alloc_branch_id(ObExecContext &exec_ctx, const int64_t count, int16_t &branch_id) +{ + int ret = OB_SUCCESS; + ObSQLSessionInfo *session = GET_MY_SESSION(exec_ctx); + transaction::ObTxDesc *tx_desc = NULL; + CK (OB_NOT_NULL(session)); + CK (OB_NOT_NULL(tx_desc = session->get_tx_desc())); + OZ (tx_desc->alloc_branch_id(count, branch_id)); + return ret; +} + }/* ns sql*/ }/* ns oceanbase */ diff --git a/src/sql/ob_sql_trans_control.h b/src/sql/ob_sql_trans_control.h index 0a853c8b54..24e10db7ed 100644 --- a/src/sql/ob_sql_trans_control.h +++ b/src/sql/ob_sql_trans_control.h @@ -206,6 +206,7 @@ public: transaction::ObTransService* txs, const int64_t nested_level); static int end_stmt(ObExecContext &exec_ctx, const bool is_rollback); + static int alloc_branch_id(ObExecContext &exec_ctx, const int64_t count, int16_t &branch_id); static int kill_query_session(ObSQLSessionInfo &session, const ObSQLSessionState &status); static int kill_tx(ObSQLSessionInfo *session, int cause); static int kill_idle_timeout_tx(ObSQLSessionInfo *session); diff --git a/src/sql/optimizer/ob_insert_log_plan.cpp b/src/sql/optimizer/ob_insert_log_plan.cpp index 131f8b68bb..b738183536 100644 --- a/src/sql/optimizer/ob_insert_log_plan.cpp +++ b/src/sql/optimizer/ob_insert_log_plan.cpp @@ -725,11 +725,6 @@ int ObInsertLogPlan::check_insert_location_need_multi_partition_dml(ObLogicalOpe } else { is_multi_part_dml = true; } - } else if (insert_stmt->is_insert_up() || insert_stmt->is_replace()) { - // #issue/44052024 - // force insert_up & replace use distribute op to avoid 4.0 branch rollback bug. - // should remove this condition in 4.1 - is_multi_part_dml = true; } else if (OB_FAIL(check_if_match_partition_wise_insert(*insert_sharding, top, is_partition_wise))) { @@ -799,6 +794,7 @@ int ObInsertLogPlan::check_if_match_partition_wise_insert(ObShardingInfo &target } else { is_partition_wise = is_match && !top.is_exchange_allocated(); } + LOG_TRACE("check partition wise", K(is_match), K(top.is_exchange_allocated())); return ret; } diff --git a/src/storage/access/ob_dml_param.cpp b/src/storage/access/ob_dml_param.cpp index 0c5e7ebe35..1640b81830 100644 --- a/src/storage/access/ob_dml_param.cpp +++ b/src/storage/access/ob_dml_param.cpp @@ -250,6 +250,7 @@ DEF_TO_STRING(ObDMLBaseParam) K_(write_flag), K_(spec_seq_no), K_(snapshot), + K_(branch_id), K_(check_schema_version)); J_OBJ_END(); return pos; diff --git a/src/storage/access/ob_dml_param.h b/src/storage/access/ob_dml_param.h index 7021297f60..84fc1f4b6f 100644 --- a/src/storage/access/ob_dml_param.h +++ b/src/storage/access/ob_dml_param.h @@ -185,6 +185,7 @@ struct ObDMLBaseParam encrypt_meta_legacy_(), spec_seq_no_(), snapshot_(), + branch_id_(0), direct_insert_task_id_(0), write_flag_(), check_schema_version_(true) @@ -215,6 +216,8 @@ struct ObDMLBaseParam transaction::ObTxSEQ spec_seq_no_; // transaction snapshot transaction::ObTxReadSnapshot snapshot_; + // parallel dml write branch id + int16_t branch_id_; int64_t direct_insert_task_id_; // 0 means no direct insert // write flag for inner write processing concurrent_control::ObWriteFlag write_flag_; diff --git a/src/storage/ls/ob_freezer.cpp b/src/storage/ls/ob_freezer.cpp index 010c58c2ec..31416b1927 100644 --- a/src/storage/ls/ob_freezer.cpp +++ b/src/storage/ls/ob_freezer.cpp @@ -40,7 +40,6 @@ ObFrozenMemtableInfo::ObFrozenMemtableInfo() end_scn_(share::ObScnRange::MIN_SCN), write_ref_cnt_(0), unsubmitted_cnt_(0), - unsynced_cnt_(0), current_right_boundary_(0) {} @@ -49,14 +48,12 @@ ObFrozenMemtableInfo::ObFrozenMemtableInfo(const ObTabletID &tablet_id, const SCN &end_scn, const int64_t write_ref_cnt, const int64_t unsubmitted_cnt, - const int64_t unsynced_cnt, const int64_t current_right_boundary) : tablet_id_(tablet_id), start_scn_(start_scn), end_scn_(end_scn), write_ref_cnt_(write_ref_cnt), unsubmitted_cnt_(unsubmitted_cnt), - unsynced_cnt_(unsynced_cnt), current_right_boundary_(current_right_boundary) {} @@ -72,7 +69,6 @@ void ObFrozenMemtableInfo::reset() end_scn_ = share::ObScnRange::MIN_SCN; write_ref_cnt_ = 0; unsubmitted_cnt_ = 0; - unsynced_cnt_ = 0; current_right_boundary_ = 0; } @@ -81,7 +77,6 @@ void ObFrozenMemtableInfo::set(const ObTabletID &tablet_id, const SCN &end_scn, const int64_t write_ref_cnt, const int64_t unsubmitted_cnt, - const int64_t unsynced_cnt, const int64_t current_right_boundary) { tablet_id_ = tablet_id; @@ -89,7 +84,6 @@ void ObFrozenMemtableInfo::set(const ObTabletID &tablet_id, end_scn_ = end_scn; write_ref_cnt_ = write_ref_cnt; unsubmitted_cnt_ = unsubmitted_cnt; - unsynced_cnt_ = unsynced_cnt; current_right_boundary_ = current_right_boundary; } @@ -142,7 +136,6 @@ int ObFreezerStat::add_memtable_info(const ObTabletID &tablet_id, const SCN &end_scn, const int64_t write_ref_cnt, const int64_t unsubmitted_cnt, - const int64_t unsynced_cnt, const int64_t current_right_boundary) { int ret = OB_SUCCESS; @@ -154,7 +147,6 @@ int ObFreezerStat::add_memtable_info(const ObTabletID &tablet_id, end_scn, write_ref_cnt, unsubmitted_cnt, - unsynced_cnt, current_right_boundary); if (OB_FAIL(memtables_info_.push_back(memtable_info))) { TRANS_LOG(WARN, "fail to push_back memtable_info", K(ret), K(tablet_id)); @@ -1288,6 +1280,20 @@ int ObFreezer::handle_memtable_for_tablet_freeze(memtable::ObIMemtable *imemtabl return ret; } +namespace { + struct FreezeDiagnoseInfo { + const char *fmt_; + const int ret_; + FreezeDiagnoseInfo(const char *fmt, const int ret): fmt_(fmt), ret_(ret) {} + DECLARE_TO_STRING + { + int64_t pos = 0; + BUF_PRINTF(fmt_, ObCurTraceId::get_trace_id_str(), ret_); + return pos; + } + }; +} + int ObFreezer::submit_log_for_freeze(bool is_try) { int ret = OB_SUCCESS; @@ -1295,7 +1301,11 @@ int ObFreezer::submit_log_for_freeze(bool is_try) share::ObLSID ls_id = get_ls_id(); const int64_t start = ObTimeUtility::current_time(); ObTabletID tablet_id(INT64_MAX); // used for diagnose - + bool trace_id_need_reset = false; + if (!ObCurTraceId::get_trace_id()->is_valid()) { + ObCurTraceId::init(GCONF.self_addr_); + bool trace_id_need_reset = true; + } do { ret = OB_SUCCESS; transaction::ObTransID fail_tx_id; @@ -1304,9 +1314,10 @@ int ObFreezer::submit_log_for_freeze(bool is_try) const int64_t cost_time = ObTimeUtility::current_time() - start; if (cost_time > 1000 * 1000) { if (TC_REACH_TIME_INTERVAL(5 * 1000 * 1000)) { - TRANS_LOG(WARN, "[Freezer] failed to traverse trans ctx to submit redo log", K(ret), + TRANS_LOG(WARN, "[Freezer] traverse_trans_to_submit_redo_log failed", K(ret), K(ls_id), K(cost_time), K(fail_tx_id)); - stat_.add_diagnose_info("traverse_trans_to_submit_redo_log failed"); + FreezeDiagnoseInfo diagnose("traverse_trans_to_submit_redo_log failed, traceId:%s, errorCode:%d", ret); + stat_.add_diagnose_info(to_cstring(diagnose)); if (OB_TMP_FAIL(ADD_SUSPECT_INFO(MINI_MERGE, ObDiagnoseTabletType::TYPE_MINI_MERGE, ls_id, tablet_id, ObSuspectInfoType::SUSPECT_SUBMIT_LOG_FOR_FREEZE, static_cast(ret), fail_tx_id.get_id()))) { @@ -1331,7 +1342,9 @@ int ObFreezer::submit_log_for_freeze(bool is_try) stat_.set_state(ObFreezeState::WAIT_READY_FOR_FLUSH); } - + if (trace_id_need_reset) { + ObCurTraceId::reset(); + } return ret; } diff --git a/src/storage/ls/ob_freezer.h b/src/storage/ls/ob_freezer.h index 3512e7787d..9badea0acc 100644 --- a/src/storage/ls/ob_freezer.h +++ b/src/storage/ls/ob_freezer.h @@ -113,7 +113,6 @@ public: const share::SCN &end_scn, const int64_t write_ref_cnt, const int64_t unsubmitted_cnt, - const int64_t unsynced_cnt, const int64_t current_right_boundary); ~ObFrozenMemtableInfo(); @@ -123,7 +122,6 @@ public: const share::SCN &end_scn, const int64_t write_ref_cnt, const int64_t unsubmitted_cnt, - const int64_t unsynced_cnt, const int64_t current_right_boundary); bool is_valid(); @@ -133,10 +131,9 @@ public: share::SCN end_scn_; int64_t write_ref_cnt_; int64_t unsubmitted_cnt_; - int64_t unsynced_cnt_; int64_t current_right_boundary_; TO_STRING_KV(K_(tablet_id), K_(start_scn), K_(end_scn), K_(write_ref_cnt), - K_(unsubmitted_cnt), K_(unsynced_cnt), K_(current_right_boundary)); + K_(unsubmitted_cnt), K_(current_right_boundary)); }; class ObFreezerStat @@ -157,7 +154,6 @@ public: const share::SCN &end_scn, const int64_t write_ref_cnt, const int64_t unsubmitted_cnt, - const int64_t unsynced_cnt, const int64_t current_right_boundary); int remove_memtable_info(const ObTabletID &tablet_id); int get_memtables_info(common::ObSArray &memtables_info); diff --git a/src/storage/ls/ob_ls_tx_service.cpp b/src/storage/ls/ob_ls_tx_service.cpp index 788a6ae9f3..59039a1148 100644 --- a/src/storage/ls/ob_ls_tx_service.cpp +++ b/src/storage/ls/ob_ls_tx_service.cpp @@ -385,8 +385,8 @@ int ObLSTxService::replay(const void *buffer, LOG_WARN("log base header deserialize error", K(ret)); } else if (OB_FAIL(ObTxReplayExecutor::execute(parent_, this, log_buf, nbytes, tmp_pos, lsn, scn, - base_header.get_replay_hint(), - ls_id_, parent_->get_tenant_id()))) { + base_header, + ls_id_))) { LOG_WARN("replay tx log error", K(ret), K(lsn), K(scn)); } return ret; @@ -518,7 +518,7 @@ int ObLSTxService::flush(SCN &recycle_scn) // only flush the common_checkpoint that whose clog need recycle if (OB_NOT_NULL(common_checkpoints_[i]) && recycle_scn >= common_checkpoints_[i]->get_rec_scn()) { if (OB_SUCCESS != (tmp_ret = common_checkpoints_[i]->flush(recycle_scn))) { - TRANS_LOG(WARN, "obCommonCheckpoint flush failed", K(tmp_ret), K(common_checkpoints_[i])); + TRANS_LOG(WARN, "obCommonCheckpoint flush failed", K(tmp_ret), K(i), K(common_checkpoints_[i])); } } } diff --git a/src/storage/memtable/mvcc/ob_multi_version_iterator.cpp b/src/storage/memtable/mvcc/ob_multi_version_iterator.cpp index f0714ddb3f..489edd88dd 100644 --- a/src/storage/memtable/mvcc/ob_multi_version_iterator.cpp +++ b/src/storage/memtable/mvcc/ob_multi_version_iterator.cpp @@ -152,6 +152,12 @@ int ObMultiVersionValueIterator::get_next_uncommitted_node( TRANS_LOG(WARN, "current trans node has not submit clog yet", K(ret), KPC_(version_iter)); } else if (NDT_COMPACT == version_iter_->type_) { // ignore compact node version_iter_ = version_iter_->prev_; + } else if (version_iter_->scn_ > merge_scn_) { + // skip tx node which not log succ + if (REACH_TIME_INTERVAL(100_ms)) { + TRANS_LOG(INFO, "skip txn-node log sync failed", KPC(version_iter_), K(merge_scn_)); + } + version_iter_ = version_iter_->prev_; } else { bool need_get_state = version_iter_->get_tx_end_scn() > merge_scn_; if (need_get_state) { diff --git a/src/storage/memtable/mvcc/ob_mvcc.h b/src/storage/memtable/mvcc/ob_mvcc.h index 3aaf35af9d..06a9a8ff37 100644 --- a/src/storage/memtable/mvcc/ob_mvcc.h +++ b/src/storage/memtable/mvcc/ob_mvcc.h @@ -23,22 +23,32 @@ class ObTransCallbackList; class ObITransCallbackIterator; class ObIMemtable; enum class MutatorType; - +struct TxChecksum : public common::ObBatchChecksum { + TxChecksum(): cnt_(0), scn_() {} + void reset() { + common::ObBatchChecksum::reset(); + cnt_ = 0; + scn_.reset(); + } + int cnt_; + share::SCN scn_; + TO_STRING_KV(K_(cnt), K_(scn), "checksum", const_cast(this)->calc()); +}; class ObITransCallback { friend class ObTransCallbackList; friend class ObITransCallbackIterator; public: - ObITransCallback() - : need_fill_redo_(true), + ObITransCallback() : need_submit_log_(true), scn_(share::SCN::max_scn()), + epoch_(0), prev_(NULL), next_(NULL) {} - ObITransCallback(const bool need_fill_redo, const bool need_submit_log) - : need_fill_redo_(need_fill_redo), + ObITransCallback(const bool need_submit_log) : need_submit_log_(need_submit_log), scn_(share::SCN::max_scn()), + epoch_(0), prev_(NULL), next_(NULL) {} virtual ~ObITransCallback() {} @@ -52,27 +62,23 @@ public: virtual transaction::ObTxSEQ get_seq_no() const { return transaction::ObTxSEQ::INVL(); } virtual int del() { return remove(); } virtual bool is_need_free() const { return true; } - virtual bool log_synced() const { return false; } void set_scn(const share::SCN scn); share::SCN get_scn() const; + bool is_log_submitted() const { return !scn_.is_max(); } + void set_epoch(int64_t epoch) { epoch_ = epoch; } + int64_t get_epoch() const { return epoch_; } int before_append_cb(const bool is_replay); void after_append_cb(const bool is_replay); - // interface for redo log generator - bool need_fill_redo() const { return need_fill_redo_; } bool need_submit_log() const { return need_submit_log_; } virtual bool is_logging_blocked() const { return false; } - int log_submitted_cb(); - int undo_log_submitted_cb(); - int log_sync_cb(const share::SCN scn); - int log_sync_fail_cb(); + int log_submitted_cb(const share::SCN scn, ObIMemtable *&last_mt); + int log_sync_fail_cb(const share::SCN scn); // interface should be implement by subclasses virtual int before_append(const bool is_replay) { return common::OB_SUCCESS; } virtual void after_append(const bool is_replay) {} - virtual int log_submitted() { return common::OB_SUCCESS; } - virtual int undo_log_submitted() { return common::OB_SUCCESS; } - virtual int log_sync(const share::SCN scn) + virtual int log_submitted(const share::SCN scn, ObIMemtable *&last_mt) { UNUSED(scn); return common::OB_SUCCESS; } - virtual int log_sync_fail() + virtual int log_sync_fail(const share::SCN max_committed_scn) { return common::OB_SUCCESS; } virtual int64_t get_data_size() { return 0; } virtual MutatorType get_mutator_type() const; @@ -104,7 +110,7 @@ public: // execution the checksum if checksum_scn is smaller or equal than your // scn. virtual int calc_checksum(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) + TxChecksum *checksumer) { UNUSED(checksum_scn); UNUSED(checksumer); @@ -152,10 +158,11 @@ protected: int before_append(ObITransCallback *node); int remove(); struct { - bool need_fill_redo_ : 1; // Identifies whether log is needed - bool need_submit_log_ : 1; // Identifies whether log has been submitted + // Identifies whether log has been submitted, and may not been synced + bool need_submit_log_ : 1; }; share::SCN scn_; + int64_t epoch_; public: int64_t owner_; private: diff --git a/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp b/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp index 39ab5538d7..7246ce252f 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp @@ -76,7 +76,8 @@ int ObIMvccCtx::register_row_commit_cb( TRANS_LOG(WARN, "alloc row callback failed", K(ret)); } else { //统计当前trans_node占用的内存大小 - add_trans_mem_total_size(data_size); + // not used now, is hotspot in pdml, so comment out + // add_trans_mem_total_size(data_size); cb->set(key, node, data_size, @@ -219,7 +220,6 @@ int ObIMvccCtx::register_table_lock_replay_cb( TRANS_LOG(WARN, "register tablelock callback failed", K(ret), KPC(lock_op)); } else { cb->set_scn(scn); - update_max_submitted_seq_no(cb->get_seq_no()); TRANS_LOG(DEBUG, "replay register table lock callback", K(*cb)); } return ret; @@ -280,11 +280,12 @@ ObMvccWriteGuard::~ObMvccWriteGuard() { if (NULL != ctx_) { int ret = OB_SUCCESS; - auto tx_ctx = ctx_->get_trans_ctx(); + transaction::ObPartTransCtx *tx_ctx = ctx_->get_trans_ctx(); ctx_->write_done(); if (OB_NOT_NULL(memtable_)) { bool is_freeze = memtable_->is_frozen_memtable(); - if (OB_FAIL(tx_ctx->submit_redo_log(is_freeze))) { + ret = tx_ctx->submit_redo_after_write(is_freeze/*force*/, write_seq_no_); + if (OB_FAIL(ret)) { if (REACH_TIME_INTERVAL(100 * 1000)) { TRANS_LOG(WARN, "failed to submit log if neccesary", K(ret), K(is_freeze), KPC(tx_ctx)); } @@ -299,7 +300,7 @@ ObMvccWriteGuard::~ObMvccWriteGuard() int ObMvccWriteGuard::write_auth(storage::ObStoreCtx &store_ctx) { int ret = common::OB_SUCCESS; - auto mem_ctx = store_ctx.mvcc_acc_ctx_.mem_ctx_; + ObMemtableCtx *mem_ctx = store_ctx.mvcc_acc_ctx_.mem_ctx_; if (!store_ctx.mvcc_acc_ctx_.is_write()) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "store_ctx was not prepared for write", K(ret), K(store_ctx)); @@ -308,6 +309,7 @@ int ObMvccWriteGuard::write_auth(storage::ObStoreCtx &store_ctx) K(exclusive_), K(store_ctx), KPC(mem_ctx)); } else { ctx_ = mem_ctx; + write_seq_no_ = store_ctx.mvcc_acc_ctx_.tx_scn_; } return ret; } diff --git a/src/storage/memtable/mvcc/ob_mvcc_ctx.h b/src/storage/memtable/mvcc/ob_mvcc_ctx.h index b8106b41d1..8d2205e29b 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_ctx.h +++ b/src/storage/memtable/mvcc/ob_mvcc_ctx.h @@ -66,8 +66,6 @@ public: trans_version_(share::SCN::max_scn()), commit_version_(share::SCN::min_scn()), lock_start_time_(0), - redo_scn_(share::SCN::min_scn()), - redo_log_id_(0), lock_wait_start_ts_(0), replay_compact_version_(share::SCN::min_scn()) { @@ -98,14 +96,11 @@ public: // for mvcc engine invoke virtual void on_wlock_retry(const ObMemtableKey& key, const transaction::ObTransID &conflict_tx_id) = 0; virtual void inc_truncate_cnt() = 0; virtual void add_trans_mem_total_size(const int64_t size) = 0; - virtual void update_max_submitted_seq_no(const transaction::ObTxSEQ seq_no) = 0; virtual transaction::ObTransID get_tx_id() const = 0; virtual transaction::ObPartTransCtx *get_trans_ctx() const = 0; // statics maintainness for txn logging virtual void inc_unsubmitted_cnt() = 0; virtual void dec_unsubmitted_cnt() = 0; - virtual void inc_unsynced_cnt() = 0; - virtual void dec_unsynced_cnt() = 0; virtual share::SCN get_tx_end_scn() const { return share::SCN::max_scn(); }; public: inline int get_alloc_type() const { return alloc_type_; } @@ -144,18 +139,16 @@ public: inline int64_t get_lock_start_time() { return lock_start_time_; } inline void set_for_replay(const bool for_replay) { trans_mgr_.set_for_replay(for_replay); } inline bool is_for_replay() const { return trans_mgr_.is_for_replay(); } - inline void set_redo_scn(const share::SCN redo_scn) { redo_scn_ = redo_scn; } - inline void set_redo_log_id(const int64_t redo_log_id) { redo_log_id_ = redo_log_id; } - inline share::SCN get_redo_scn() const { return redo_scn_; } - inline int64_t get_redo_log_id() const { return redo_log_id_; } inline void set_lock_wait_start_ts(const int64_t lock_wait_start_ts) { lock_wait_start_ts_ = lock_wait_start_ts; } share::SCN get_replay_compact_version() const { return replay_compact_version_; } void set_replay_compact_version(const share::SCN v) { replay_compact_version_ = v; } inline int64_t get_lock_wait_start_ts() const { return lock_wait_start_ts_; } - void acquire_callback_list() { trans_mgr_.acquire_callback_list(); } + int acquire_callback_list(const bool new_epoch, const bool need_merge) + { return trans_mgr_.acquire_callback_list(new_epoch, need_merge); } void revert_callback_list() { trans_mgr_.revert_callback_list(); } - + int get_tx_seq_replay_idx(const transaction::ObTxSEQ seq) const + { return trans_mgr_.get_tx_seq_replay_idx(seq); } int register_row_commit_cb( const ObMemtableKey *key, ObMvccRow *value, @@ -192,8 +185,6 @@ public: trans_version_ = share::SCN::max_scn(); commit_version_ = share::SCN::min_scn(); lock_start_time_ = 0; - redo_scn_ = share::SCN::min_scn(); - redo_log_id_ = 0; lock_wait_start_ts_ = 0; replay_compact_version_ = share::SCN::min_scn(); } @@ -245,8 +236,6 @@ protected: share::SCN trans_version_; share::SCN commit_version_; int64_t lock_start_time_; - share::SCN redo_scn_; - int64_t redo_log_id_; int64_t lock_wait_start_ts_; share::SCN replay_compact_version_; }; @@ -258,7 +247,9 @@ public: ObMvccWriteGuard(const bool exclusive = false) : exclusive_(exclusive), ctx_(NULL), - memtable_(NULL) {} + memtable_(NULL), + write_seq_no_() + {} ~ObMvccWriteGuard(); void set_memtable(ObMemtable *memtable) { memtable_ = memtable; @@ -277,6 +268,7 @@ private: const bool exclusive_; // if true multiple write_auth will be serialized ObMemtableCtx *ctx_; ObMemtable *memtable_; + transaction::ObTxSEQ write_seq_no_; }; } } diff --git a/src/storage/memtable/mvcc/ob_mvcc_define.h b/src/storage/memtable/mvcc/ob_mvcc_define.h index 1274a6e334..f0f42aa5ce 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_define.h +++ b/src/storage/memtable/mvcc/ob_mvcc_define.h @@ -27,6 +27,8 @@ class ObRowData; // Arguments for building tx node struct ObTxNodeArg { + // trans id + transaction::ObTransID tx_id_; // data_ is the new row of the modifiction const ObMemtableData *data_; // old_row_ is the old row of the modificattion @@ -46,7 +48,8 @@ struct ObTxNodeArg share::SCN scn_; int64_t column_cnt_; - TO_STRING_KV(KP_(data), + TO_STRING_KV(K_(tx_id), + KP_(data), KP_(old_row), K_(modify_count), K_(acc_checksum), @@ -56,12 +59,14 @@ struct ObTxNodeArg K_(column_cnt)); // Constructor for leader - ObTxNodeArg(const ObMemtableData *data, + ObTxNodeArg(const transaction::ObTransID tx_id, + const ObMemtableData *data, const ObRowData *old_row, const int64_t memstore_version, const transaction::ObTxSEQ seq_no, const int64_t column_cnt) - : data_(data), + : tx_id_(tx_id), + data_(data), old_row_(old_row), modify_count_(UINT32_MAX), acc_checksum_(0), @@ -71,7 +76,8 @@ struct ObTxNodeArg column_cnt_(column_cnt) {} // Constructor for follower - ObTxNodeArg(const ObMemtableData *data, + ObTxNodeArg(const transaction::ObTransID tx_id, + const ObMemtableData *data, const ObRowData *old_row, const int64_t memstore_version, const transaction::ObTxSEQ seq_no, @@ -79,7 +85,8 @@ struct ObTxNodeArg const uint32_t acc_checksum, const share::SCN scn, const int64_t column_cnt) - : data_(data), + : tx_id_(tx_id), + data_(data), old_row_(old_row), modify_count_(modify_count), acc_checksum_(acc_checksum), @@ -89,6 +96,7 @@ struct ObTxNodeArg column_cnt_(column_cnt) {} void reset() { + tx_id_.reset(); data_ = NULL; old_row_ = NULL; modify_count_ = 0; diff --git a/src/storage/memtable/mvcc/ob_mvcc_engine.cpp b/src/storage/memtable/mvcc/ob_mvcc_engine.cpp index df549c857e..92f6df00b8 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_engine.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_engine.cpp @@ -317,7 +317,7 @@ int ObMvccEngine::mvcc_write(ObIMemtableCtx &ctx, int ret = OB_SUCCESS; ObMvccTransNode *node = NULL; - if (OB_FAIL(build_tx_node_(ctx, arg, node))) { + if (OB_FAIL(build_tx_node_(arg, node))) { TRANS_LOG(WARN, "build tx node failed", K(ret), K(ctx), K(arg)); } else if (OB_FAIL(value.mvcc_write(ctx, write_flag, @@ -335,34 +335,29 @@ int ObMvccEngine::mvcc_write(ObIMemtableCtx &ctx, return ret; } -int ObMvccEngine::mvcc_replay(ObIMemtableCtx &ctx, - const ObMemtableKey *stored_key, - ObMvccRow &value, - const ObTxNodeArg &arg, +int ObMvccEngine::mvcc_replay(const ObTxNodeArg &arg, ObMvccReplayResult &res) { int ret = OB_SUCCESS; ObMvccTransNode *node = NULL; - if (OB_FAIL(build_tx_node_(ctx, arg, node))) { - TRANS_LOG(WARN, "build tx node failed", K(ret), K(ctx), K(arg)); + if (OB_FAIL(build_tx_node_(arg, node))) { + TRANS_LOG(WARN, "build tx node failed", K(ret), K(arg)); } else { res.tx_node_ = node; - TRANS_LOG(DEBUG, "mvcc replay succeed", K(ret), K(ctx), K(arg)); + TRANS_LOG(DEBUG, "mvcc replay succeed", K(ret), K(arg)); } return ret; } -int ObMvccEngine::build_tx_node_(ObIMemtableCtx &ctx, - const ObTxNodeArg &arg, +int ObMvccEngine::build_tx_node_(const ObTxNodeArg &arg, ObMvccTransNode *&node) { int ret = OB_SUCCESS; - if (OB_FAIL(kv_builder_->dup_data(node, *engine_allocator_, arg.data_))) { TRANS_LOG(WARN, "MvccTranNode dup fail", K(ret), "node", node); } else { - node->tx_id_ = ctx.get_tx_id(); + node->tx_id_ = arg.tx_id_; node->trans_version_ = SCN::max_scn(); node->modify_count_ = arg.modify_count_; node->acc_checksum_ = arg.acc_checksum_; @@ -391,7 +386,6 @@ int ObMvccEngine::ensure_kv(const ObMemtableKey *stored_key, TRANS_LOG(WARN, "ensure_row fail", K(ret)); } } - return ret; } diff --git a/src/storage/memtable/mvcc/ob_mvcc_engine.h b/src/storage/memtable/mvcc/ob_mvcc_engine.h index 963073768d..e9a5f1a843 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_engine.h +++ b/src/storage/memtable/mvcc/ob_mvcc_engine.h @@ -78,12 +78,8 @@ public: // and always succeed. void mvcc_undo(ObMvccRow *value); - // mvcc_replay builds the ObMvccTransNode according to the arg and replay - // into the ascending ordering of the value based on the scn recorded in ctx. - int mvcc_replay(ObIMemtableCtx &ctx, - const ObMemtableKey *stored_key, - ObMvccRow &value, - const ObTxNodeArg &arg, + // mvcc_replay builds the ObMvccTransNode according to the arg + int mvcc_replay(const ObTxNodeArg &arg, ObMvccReplayResult &res); // ensure_kv is used to make sure b-tree is no longer broken by the deleted @@ -120,8 +116,7 @@ private: int try_compact_row_when_mvcc_read_(const share::SCN &snapshot_version, ObMvccRow &row); - int build_tx_node_(ObIMemtableCtx &ctx, - const ObTxNodeArg &arg, + int build_tx_node_(const ObTxNodeArg &arg, ObMvccTransNode *&node); private: DISALLOW_COPY_AND_ASSIGN(ObMvccEngine); diff --git a/src/storage/memtable/mvcc/ob_mvcc_row.cpp b/src/storage/memtable/mvcc/ob_mvcc_row.cpp index 617daf2eb2..fbde7c8657 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_row.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_row.cpp @@ -222,7 +222,7 @@ int64_t ObMvccTransNode::to_string(char *buf, const int64_t buf_len) const "snapshot_barrier=%ld " "snapshot_barrier_flag=%ld " "mtd=%s " - "seq_no=%ld", + "seq_no=%s", this, to_cstring(trans_version_), to_cstring(scn_), @@ -238,7 +238,7 @@ int64_t ObMvccTransNode::to_string(char *buf, const int64_t buf_len) const & (~SNAPSHOT_VERSION_BARRIER_BIT), snapshot_version_barrier_ >> 62, to_cstring(*mtd), - seq_no_.cast_to_int()); + to_cstring(seq_no_)); return pos; } @@ -573,6 +573,11 @@ int ObMvccRow::insert_trans_node(ObIMvccCtx &ctx, ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "meet unexpected index_node", KR(ret), K(*prev), K(node), K(*index_node), K(*this)); abort_unless(0); + } else if (prev->tx_id_ == node.tx_id_ && OB_UNLIKELY(prev->seq_no_ > node.seq_no_)) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "prev node seq_no > this node", KR(ret), KPC(prev), K(node), KPC(this)); + usleep(1000); + ob_abort(); } else { next_node = next; ATOMIC_STORE(&(node.next_), next); @@ -609,6 +614,14 @@ int ObMvccRow::insert_trans_node(ObIMvccCtx &ctx, tmp = ATOMIC_LOAD(prev); } } + if (OB_SUCC(ret) && OB_NOT_NULL(tmp) && tmp->tx_id_ == node.tx_id_) { + if (OB_UNLIKELY(tmp->seq_no_ > node.seq_no_)) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "prev node seq_no > this node", KR(ret), "prev", PC(tmp), K(node), KPC(this)); + usleep(1000); + ob_abort(); + } + } if (OB_SUCC(ret)) { ATOMIC_STORE(&(node.prev_), tmp); ATOMIC_STORE(prev, &node); @@ -952,6 +965,7 @@ int ObMvccRow::mvcc_write_(ObIMemtableCtx &ctx, return ret; } +__attribute__((noinline)) int ObMvccRow::check_double_insert_(const SCN snapshot_version, ObMvccTransNode &node, ObMvccTransNode *prev) @@ -996,7 +1010,6 @@ int ObMvccRow::mvcc_write(ObIMemtableCtx &ctx, { int ret = OB_SUCCESS; const SCN snapshot_version = snapshot.version_; - if (max_trans_version_.atomic_load() > snapshot_version || max_elr_trans_version_.atomic_load() > snapshot_version) { // Case 3. successfully locked while tsc @@ -1026,7 +1039,6 @@ int ObMvccRow::mvcc_write(ObIMemtableCtx &ctx, (void)mvcc_undo(); } } - return ret; } diff --git a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp index c5ab1fea98..e02ccaf654 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp @@ -18,10 +18,12 @@ #include "storage/memtable/ob_memtable_context.h" #include "storage/memtable/ob_memtable_data.h" #include "storage/memtable/ob_memtable_util.h" +#include "storage/memtable/ob_memtable_mutator.h" #include "lib/atomic/atomic128.h" #include "storage/memtable/ob_lock_wait_mgr.h" #include "storage/tx/ob_trans_ctx.h" #include "storage/tx/ob_trans_part_ctx.h" +#include "storage/tx/ob_tx_stat.h" #include "ob_mvcc_ctx.h" #include "storage/memtable/ob_memtable_interface.h" @@ -100,7 +102,6 @@ int ObITransCallback::before_append_cb(const bool is_replay) { int ret = before_append(is_replay); if (OB_SUCC(ret)) { - need_fill_redo_ = !is_replay; need_submit_log_ = !is_replay; } return ret; @@ -111,52 +112,21 @@ void ObITransCallback::after_append_cb(const bool is_replay) (void)after_append(is_replay); } -int ObITransCallback::log_submitted_cb() +int ObITransCallback::log_submitted_cb(const SCN scn, ObIMemtable *&last_mt) { int ret = OB_SUCCESS; if (need_submit_log_) { - if (OB_SUCC(log_submitted())) { + if (OB_SUCC(log_submitted(scn, last_mt))) { + set_scn(scn); need_submit_log_ = false; } } return ret; } -int ObITransCallback::undo_log_submitted_cb() +int ObITransCallback::log_sync_fail_cb(const share::SCN max_committed_scn) { - int ret = OB_SUCCESS; - if (need_submit_log_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "log has not beed submitted", K(ret), K(*this)); - } else if (!need_fill_redo_) { - } else if (OB_SUCC(undo_log_submitted())) { - need_submit_log_ = true; - } - return ret; -} - -int ObITransCallback::log_sync_cb(const SCN scn) -{ - int ret = OB_SUCCESS; - if (!need_fill_redo_) { - } else if (OB_UNLIKELY(SCN::max_scn() == scn)) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "log ts should not be invalid", K(ret), K(scn), K(*this)); - } else if (OB_SUCC(log_sync(scn))) { - need_fill_redo_ = false; - } - return ret; -} - -int ObITransCallback::log_sync_fail_cb() -{ - int ret = OB_SUCCESS; - if (need_fill_redo_) { - if (OB_SUCC(log_sync_fail())) { - need_fill_redo_ = false; - } - } - return ret; + return log_sync_fail(max_committed_scn); } // All safety check is in before append @@ -208,13 +178,23 @@ ObTransCallbackMgr::RDLockGuard::RDLockGuard(const SpinRWLock &rwlock) } #endif +#define CALLBACK_LISTS_FOREACH_(idx, list, CONST) \ + CONST ObTxCallbackList *list = &callback_list_; \ + const int list_cnt = (!need_merge_ && callback_lists_) ? MAX_CALLBACK_LIST_COUNT : 1; \ + for (int idx = 0; OB_SUCC(ret) && idx < list_cnt; \ + list = (list_cnt > 1 ? callback_lists_ + idx : NULL), ++idx) + +#define CALLBACK_LISTS_FOREACH_CONST(idx, list) CALLBACK_LISTS_FOREACH_(idx, list, const) +#define CALLBACK_LISTS_FOREACH(idx, list) CALLBACK_LISTS_FOREACH_(idx, list,) + void ObTransCallbackMgr::reset() { int64_t stat = ATOMIC_LOAD(¶llel_stat_); - + skip_checksum_ = false; callback_list_.reset(); - if (PARALLEL_STMT == stat && NULL != callback_lists_) { - for (int i = 0; i < MAX_CALLBACK_LIST_COUNT; ++i) { + if (callback_lists_) { + int cnt = need_merge_ ? MAX_CALLBACK_LIST_COUNT : MAX_CALLBACK_LIST_COUNT - 1; + for (int i = 0; i < cnt; ++i) { if (!callback_lists_[i].empty()) { ob_abort(); TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "txn callback list is broken", K(stat), K(i), K(this)); @@ -235,6 +215,12 @@ void ObTransCallbackMgr::reset() cb_allocators_ = NULL; } parallel_stat_ = 0; + write_epoch_ = 0; + write_epoch_start_tid_ = 0; + need_merge_ = false; + for_replay_ = false; + serial_final_scn_.set_max(); + serial_sync_scn_.set_min(); callback_main_list_append_count_ = 0; callback_slave_list_append_count_ = 0; callback_slave_list_merge_count_ = 0; @@ -269,7 +255,7 @@ void *ObTransCallbackMgr::callback_alloc(const int64_t size) const int64_t slot = tid % MAX_CB_ALLOCATOR_COUNT; int64_t stat = ATOMIC_LOAD(¶llel_stat_); - if (PARALLEL_STMT == stat) { + if (PARALLEL_STMT == stat || (for_replay_ && parallel_replay_)) { if (NULL == cb_allocators_) { WRLockGuard guard(rwlock_); if (NULL == cb_allocators_) { @@ -317,49 +303,77 @@ void *ObTransCallbackMgr::callback_alloc(const int64_t size) return callback; } +inline +int ObTransCallbackMgr::extend_callback_lists_(const int16_t cnt) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(NULL == callback_lists_)) { + WRLockGuard guard(rwlock_); + if (NULL == callback_lists_) { + int size = sizeof(ObTxCallbackList) * cnt; + ObTxCallbackList *tmp_callback_lists = (ObTxCallbackList *)cb_allocator_.alloc(size); + if (OB_ISNULL(tmp_callback_lists)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + TRANS_LOG(WARN, "alloc cb lists fail", K(ret)); + } else { + for (int16_t i = 0; i < cnt; ++i) { + UNUSED(new(tmp_callback_lists + i) ObTxCallbackList(*this, i+1)); + } + ATOMIC_STORE(&callback_lists_, tmp_callback_lists); + } + } + } + return ret; +} + +int ObTransCallbackMgr::get_tx_seq_replay_idx(const transaction::ObTxSEQ seq) const +{ + return seq.get_branch() % MAX_CALLBACK_LIST_COUNT; +} + +_RLOCAL(bool, ObTransCallbackMgr::parallel_replay_); + +// called by write and replay: int ObTransCallbackMgr::append(ObITransCallback *node) { int ret = OB_SUCCESS; - const int64_t tid = get_itid() + 1; - const int64_t slot = tid % MAX_CALLBACK_LIST_COUNT; - int64_t stat = ATOMIC_LOAD(¶llel_stat_); - (void)before_append(node); - - if (PARALLEL_STMT == stat) { - if (NULL == callback_lists_) { - WRLockGuard guard(rwlock_); - if (NULL == callback_lists_) { - ObTxCallbackList *tmp_callback_lists = NULL; - if (NULL == (tmp_callback_lists = (ObTxCallbackList *)cb_allocator_.alloc( - sizeof(ObTxCallbackList) * MAX_CALLBACK_LIST_COUNT))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - TRANS_LOG(WARN, "alloc cb lists fail", K(ret)); - } else { - for (int i = 0; i < MAX_CALLBACK_LIST_COUNT; ++i) { - UNUSED(new(tmp_callback_lists + i) ObTxCallbackList(*this)); - } - callback_lists_ = tmp_callback_lists; - } - } - } - - if (OB_SUCC(ret)) { - if (NULL == callback_lists_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "callback lists is not inited", K(ret)); - } else { - ret = callback_lists_[slot].append_callback(node, for_replay_); - add_slave_list_append_cnt(); - } + if (!for_replay_) { + node->set_epoch(write_epoch_); + } + const transaction::ObTxSEQ seq_no = node->get_seq_no(); + if (seq_no.support_branch()) { + // NEW since version 4.3, select by branch + int slot = seq_no.get_branch() % MAX_CALLBACK_LIST_COUNT; + if (slot == 0) { + // no parallel and no branch requirement + ret = callback_list_.append_callback(node, for_replay_, parallel_replay_, is_serial_final_()); + // try to extend callback_lists_ if required + } else if (!callback_lists_ && OB_FAIL(extend_callback_lists_(MAX_CALLBACK_LIST_COUNT - 1))) { + TRANS_LOG(WARN, "extend callback lists failed", K(ret)); + } else { + ret = callback_lists_[slot - 1].append_callback(node, for_replay_, parallel_replay_, is_serial_final_()); } } else { - ret = callback_list_.append_callback(node, for_replay_); - add_main_list_append_cnt(); + // OLD before version 4.3 + // if has parallel select from callback_lists_ + // don't select main, and merge into main finally + const int64_t tid = get_itid() + 1; + int slot = tid % MAX_CALLBACK_LIST_COUNT; + int64_t stat = ATOMIC_LOAD(¶llel_stat_); + if (PARALLEL_STMT == stat) { + if (OB_FAIL(!callback_lists_ && extend_callback_lists_(MAX_CALLBACK_LIST_COUNT))) { + TRANS_LOG(WARN, "extend callback lists failed", K(ret)); + } else { + ret = callback_lists_[slot].append_callback(node, for_replay_, parallel_replay_, true); + add_slave_list_append_cnt(); + } + } else { + ret = callback_list_.append_callback(node, for_replay_, parallel_replay_, true); + add_main_list_append_cnt(); + } } - after_append(node, ret); - return ret; } @@ -386,20 +400,42 @@ void ObTransCallbackMgr::after_append(ObITransCallback *node, const int ret_code } int ObTransCallbackMgr::rollback_to(const ObTxSEQ to_seq_no, - const ObTxSEQ from_seq_no) + const ObTxSEQ from_seq_no, + const share::SCN replay_scn) { int ret = OB_SUCCESS; - if (OB_FAIL(callback_list_.remove_callbacks_for_rollback_to(to_seq_no))) { - TRANS_LOG(WARN, "invalid argument", K(ret), K(from_seq_no), K(to_seq_no)); + int slot = -1; + if (OB_LIKELY(to_seq_no.support_branch())) { // since 4.3 + // it is a global savepoint, rollback on all list + if (to_seq_no.get_branch() == 0) { + CALLBACK_LISTS_FOREACH(idx, list) { + slot = idx; + ret = list->remove_callbacks_for_rollback_to(to_seq_no, from_seq_no, replay_scn); + } + } else { + // it is a branch level savepoint, rollback on list contains the branch + slot = to_seq_no.get_branch() % MAX_CALLBACK_LIST_COUNT; + if (slot == 0) { + ret = callback_list_.remove_callbacks_for_rollback_to(to_seq_no, from_seq_no, replay_scn); + } else if (callback_lists_) { + ret = callback_lists_[slot - 1].remove_callbacks_for_rollback_to(to_seq_no, from_seq_no, replay_scn); + } else { /*callback_lists_ is empty, no need do rollback */ } + } + } else { // before 4.3 + ret = callback_list_.remove_callbacks_for_rollback_to(to_seq_no, from_seq_no, replay_scn); + } + if (OB_FAIL(ret)) { + TRANS_LOG(WARN, "rollback to fail", K(ret), K(slot), K(from_seq_no), K(to_seq_no)); } return ret; } +// merge `callback_lists_` into `callback_list_` void ObTransCallbackMgr::merge_multi_callback_lists() { int64_t stat = ATOMIC_LOAD(¶llel_stat_); int64_t cnt = 0; - if (PARALLEL_STMT == stat) { + if (OB_UNLIKELY(ATOMIC_LOAD(&need_merge_)) && PARALLEL_STMT == stat) { WRLockGuard guard(rwlock_); if (OB_NOT_NULL(callback_lists_)) { for (int64_t i = 0; i < MAX_CALLBACK_LIST_COUNT; ++i) { @@ -416,11 +452,13 @@ void ObTransCallbackMgr::merge_multi_callback_lists() void ObTransCallbackMgr::force_merge_multi_callback_lists() { int64_t cnt = 0; - WRLockGuard guard(rwlock_); - if (OB_NOT_NULL(callback_lists_)) { - for (int64_t i = 0; i < MAX_CALLBACK_LIST_COUNT; ++i) { - cnt = callback_list_.concat_callbacks(callback_lists_[i]); - add_slave_list_merge_cnt(cnt); + if (OB_UNLIKELY(ATOMIC_LOAD(&need_merge_))) { + WRLockGuard guard(rwlock_); + if (OB_NOT_NULL(callback_lists_)) { + for (int64_t i = 0; i < MAX_CALLBACK_LIST_COUNT; ++i) { + cnt = callback_list_.concat_callbacks(callback_lists_[i]); + add_slave_list_merge_cnt(cnt); + } } } TRANS_LOG(DEBUG, "force merge callback lists to callback list", K(host_.get_tx_id())); @@ -448,92 +486,704 @@ void ObTransCallbackMgr::reset_pdml_stat() force_merge_multi_callback_lists(); } -int ObTransCallbackMgr::remove_callbacks_for_fast_commit(const ObITransCallback *generate_cursor, - bool &meet_generate_cursor) +// only for replay +// @callback_list_idx: the current replay thread replayed queue +// for serial replay, the queue maybe replay logs belongs to other callback-list +// @stop_scn: should stop at this scn, it equals to the scn of current replaying log minus 1 +// because current replaying log maybe replay failed and rolled back +int ObTransCallbackMgr::remove_callbacks_for_fast_commit(const int16_t callback_list_idx, + const share::SCN stop_scn) { int ret = OB_SUCCESS; - - if (OB_FAIL(callback_list_.remove_callbacks_for_fast_commit(generate_cursor, - meet_generate_cursor))) { - TRANS_LOG(WARN, "remove callbacks for fast commit fail", K(ret)); + RDLockGuard guard(rwlock_); + // NOTE: + // this can handle both NEW(since 4.3) and compatible with OLD version: + // because before 4.3, replay only append to main (the `callback_list_`) + if (OB_UNLIKELY(callback_list_idx != 0 || is_serial_final_())) { + ObTxCallbackList *list = get_callback_list_(callback_list_idx, true); + if (OB_ISNULL(list)) { + // the callback list may not extended by replay redo if row is skipped + } else { + // if not reach serial final, and parallel replayed log, fast commit + // should stop at serial replayed position + const bool serial_final = is_serial_final_(); + const share::SCN real_stop_scn = serial_final ? stop_scn : serial_sync_scn_; + if (OB_FAIL(list->remove_callbacks_for_fast_commit(real_stop_scn))) { + TRANS_LOG(WARN, "remove callbacks for fast commit fail", K(ret), + K(real_stop_scn), K(stop_scn), K(callback_list_idx), KPC(list)); + } + } + } else { // for serial replayed log, and not reach serial final, handle all list + if (OB_LIKELY(NULL == callback_lists_)) { + ret = callback_list_.remove_callbacks_for_fast_commit(stop_scn); + } else { + CALLBACK_LISTS_FOREACH(idx, list) { + if (OB_FAIL(list->remove_callbacks_for_fast_commit(stop_scn))) { + TRANS_LOG(WARN, "remove callbacks for fast commit fail", K(ret), K(idx), KPC(list)); + } + } + } } - return ret; } -int ObTransCallbackMgr::remove_callback_for_uncommited_txn( - const memtable::ObMemtableSet *memtable_set, - const share::SCN max_applied_scn) +// for leader +// called after log apply thread has callbacked the log_cb +// @scopes: the log's callback-list scopes +int ObTransCallbackMgr::remove_callbacks_for_fast_commit(const ObCallbackScopeArray &scopes) +{ + // this can handle both NEW (since 4.3) and OLD (before 4.3): + // before 4.3: scopes must be single and came from the main list + const share::SCN stop_scn = is_serial_final_() ? share::SCN::invalid_scn() : serial_sync_scn_; + int ret = OB_SUCCESS; + ARRAY_FOREACH(scopes, i) { + if (OB_FAIL(scopes.at(i).host_->remove_callbacks_for_fast_commit(stop_scn))) { + TRANS_LOG(WARN, "remove callbacks for fast commit fail", K(ret), K(i), KPC(scopes.at(i).host_)); + } + } + return ret; +} + +// memtable will be released, remove callbacks refer to it +// these callbacks has been logged and the writes were in checkpoint +int ObTransCallbackMgr::remove_callback_for_uncommited_txn(const memtable::ObMemtableSet *memtable_set) { int ret = OB_SUCCESS; - + const bool serial_final = is_serial_final_(); + const share::SCN stop_scn = serial_final ? share::SCN::max_scn() : serial_sync_scn_; if (OB_ISNULL(memtable_set)) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "memtable is null", K(ret)); - } else if (OB_FAIL(callback_list_.remove_callbacks_for_remove_memtable(memtable_set, max_applied_scn))) { - TRANS_LOG(WARN, "fifo remove callback fail", K(ret), KPC(memtable_set)); + } else if (need_merge_) { // OLD (before 4.3) + if (OB_FAIL(callback_list_.remove_callbacks_for_remove_memtable(memtable_set, stop_scn))) { + TRANS_LOG(WARN, "fifo remove callback fail", K(ret), KPC(memtable_set)); + } + } else { // NEW (since 4.3) + CALLBACK_LISTS_FOREACH(idx, list) { + if (OB_FAIL(list->remove_callbacks_for_remove_memtable(memtable_set, stop_scn))) { + TRANS_LOG(WARN, "fifo remove callback fail", K(ret), K(idx), KPC(memtable_set)); + } + } } return ret; } +// when leader revoked, writes has not been logged must be discarded +// otherwise freeze memtable checkpoint will be blocked on waiting these. int ObTransCallbackMgr::clean_unlog_callbacks(int64_t &removed_cnt) { int ret = OB_SUCCESS; - - if (OB_FAIL(callback_list_.clean_unlog_callbacks(removed_cnt))) { - TRANS_LOG(WARN, "clean unlog callbacks failed", K(ret)); + if (need_merge_) { // OLD (before 4.3) + if (OB_FAIL(callback_list_.clean_unlog_callbacks(removed_cnt))) { + TRANS_LOG(WARN, "clean unlog callbacks failed", K(ret)); + } + } else { // NEW (since 4.3) + CALLBACK_LISTS_FOREACH(idx, list) { + int64_t rm_cnt = 0; + if (OB_FAIL(list->clean_unlog_callbacks(rm_cnt))) { + TRANS_LOG(WARN, "clean unlog callbacks failed", K(ret), K(idx)); + } else { + removed_cnt += rm_cnt; + } + } } - + TRANS_LOG(TRACE, "clean callbacks", K(ret), K(removed_cnt)); return ret; } int ObTransCallbackMgr::calc_checksum_before_scn(const SCN scn, - uint64_t &checksum, - SCN &checksum_scn) + ObIArray &checksum, + ObIArray &checksum_scn) { int ret = OB_SUCCESS; - - if (SCN::max_scn() == scn) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "log ts is invalid", K(scn)); - } else if (OB_FAIL(callback_list_.tx_calc_checksum_before_scn(scn))) { - TRANS_LOG(WARN, "calc checksum with minor freeze failed", K(ret), K(scn)); - } else { - callback_list_.get_checksum_and_scn(checksum, checksum_scn); + const share::SCN stop_scn = is_serial_final_() ? share::SCN::max_scn() : serial_sync_scn_; + const bool is_single_callback_list = ATOMIC_LOAD(&callback_lists_) == NULL; + if (need_merge_ || is_single_callback_list) { // OLD (before 4.3) or only single callback_list + if (OB_FAIL(callback_list_.tx_calc_checksum_before_scn(stop_scn))) { + TRANS_LOG(WARN, "calc checksum fail", K(ret)); + } else { + callback_list_.get_checksum_and_scn(checksum.at(0), checksum_scn.at(0)); + } + } else { // new (since 4.3) and multiple callback_list + // reserve space + if (checksum.count() < MAX_CALLBACK_LIST_COUNT) { + if (OB_FAIL(checksum.reserve(MAX_CALLBACK_LIST_COUNT))) { + TRANS_LOG(WARN, "reserve fail", K(ret)); + } else if (OB_FAIL(checksum_scn.reserve(MAX_CALLBACK_LIST_COUNT))) { + TRANS_LOG(WARN, "reserve fail", K(ret)); + } else { + for (int i = checksum.count(); i < MAX_CALLBACK_LIST_COUNT; i++) { + checksum.push_back(0); + checksum_scn.push_back(share::SCN::min_scn()); + } + } + } + if (OB_SUCC(ret)) { + CALLBACK_LISTS_FOREACH(idx, list) { + if (OB_FAIL(list->tx_calc_checksum_before_scn(stop_scn))) { + TRANS_LOG(WARN, "calc checksum with minor freeze failed", K(ret), K(stop_scn), K(idx)); + } else { + list->get_checksum_and_scn(checksum.at(idx), checksum_scn.at(idx)); + } + } + } } - return ret; } -int ObTransCallbackMgr::sync_log_fail(const ObCallbackScope &callbacks, +#define FILL_LOG_TRACE_LEVEL TRACE +#define FILL_LOG_TRACE(fmt, ...) TRANS_LOG(FILL_LOG_TRACE_LEVEL, "[FILL_LOG]" fmt, K(ret), KPC(this), ##__VA_ARGS__) +void ObTransCallbackMgr::calc_list_fill_log_epoch_(const int list_idx, int64_t &epoch_from, int64_t &epoch_to) +{ + epoch_to = epoch_from = 0; + int64_t this_epoch = get_callback_list_(list_idx, false)->get_log_epoch(); + int64_t other_min = INT64_MAX; + int list_cnt = get_logging_list_count(); + bool no_result = false; + for (int i=0; i< list_cnt; i++) { + if (i != list_idx) { + ObTxCallbackList *list = get_callback_list_(i, false); + int64_t epoch_i = list->get_log_epoch(); + if (epoch_i < this_epoch) { + no_result = true; + TRANS_LOG(TRACE, "no_result", K(epoch_i), K(i), K(this_epoch)); + break; + } + if (epoch_i < other_min) { + other_min = epoch_i; + } + } + } + if (!no_result) { + epoch_from = this_epoch; + epoch_to = other_min; + } +} + +void ObTransCallbackMgr::calc_next_to_fill_log_info_(const ObIArray &arr, + int &index, + int64_t &epoch_from, + int64_t &epoch_to) +{ + index = -1; + epoch_to = epoch_from = INT64_MAX; + for (int i =0; i< arr.count(); i++) { + if (arr.at(i) < epoch_from) { + index = i; + epoch_to = epoch_from; + epoch_from = arr.at(i); + } else if (arr.at(i) < epoch_to) { + epoch_to = arr.at(i); + } + } +} + +int ObTransCallbackMgr::prep_and_fill_from_list_(ObTxFillRedoCtx &ctx, + ObITxFillRedoFunctor &func, + int16 &callback_scope_idx, + const int index, + int64_t epoch_from, + int64_t epoch_to) +{ + int ret = OB_SUCCESS; + // alloc callback scope for this list if not set + if (callback_scope_idx == -1) { + ObCallbackScope scope; + if (OB_FAIL(ctx.helper_->callbacks_.push_back(scope))) { + TRANS_LOG(WARN, "prepare callbackscope fail", K(ret)); + } else { + callback_scope_idx = ctx.helper_->callbacks_.count() - 1; + } + FILL_LOG_TRACE("choose callback scope idx", K(index), K(callback_scope_idx)); + } + // prepare fill ctx and do fill + if (OB_SUCC(ret)) { + ctx.list_idx_ = index; + ctx.list_ = get_callback_list_(index, false); + ctx.callback_scope_= &ctx.helper_->callbacks_[callback_scope_idx]; + ctx.callback_scope_->host_ = ctx.list_; + ObITransCallback *log_cursor = NULL; + if (ctx.callback_scope_->is_empty()) { + log_cursor = ctx.list_->get_log_cursor(); + } else { + log_cursor = *(ctx.callback_scope_->end_ + 1); + } + ctx.epoch_from_ = epoch_from; + ctx.epoch_to_ = epoch_to; + ret = ctx.list_->fill_log(log_cursor, ctx, func); + // after fill, if none is filled, reset the callback-scope for this list + if (ctx.callback_scope_->is_empty()) { + ctx.helper_->callbacks_.pop_back(); + callback_scope_idx = -1; + FILL_LOG_TRACE("fill from list result is empty, revert"); + } + ctx.callback_scope_ = NULL; + } + return ret; +} + +bool ObTransCallbackMgr::check_list_has_min_epoch_(const int my_idx, const int64_t my_epoch, int64_t &min_epoch, int &min_idx) +{ + bool ret = true; + int list_cnt = get_logging_list_count(); + for (int i=0; i< list_cnt; i++) { + if (i != my_idx) { + ObTxCallbackList *list = get_callback_list_(i, false); + int64_t epoch_i = list->get_log_epoch(); + if (epoch_i < my_epoch) { + ret = false; + min_epoch = epoch_i; + min_idx = i; + TRANS_LOG(DEBUG, "hit", K(epoch_i), K(i)); + break; + } + } + } + return ret; +} + +int ObTransCallbackMgr::get_next_flush_log_guard(ObCallbackListLogGuard &lock_guard, int &list_idx) +{ + int ret = OB_SUCCESS; + int list_cnt = get_logging_list_count(); + int64_t epoch = INT64_MAX; + int idx = -1; + ObTxCallbackList *l = NULL; + for (int i =0; i< list_cnt; i++) { + ObTxCallbackList *list = get_callback_list_(i, false); + if (list->get_log_epoch() < epoch) { + epoch = list->get_log_epoch(); + idx = i; + l = list; + } + } + common::ObByteLock *lock = NULL; + if (idx == -1) { + ret = OB_ENTRY_NOT_EXIST; + } else if (OB_ISNULL(lock = l->try_lock_log())) { + ret = OB_NEED_RETRY; + } else { + lock_guard.set(lock); + list_idx = idx; + } + return ret; +} + +// retval: +// - OB_EAGAIN: other list has small log_epoch +// - OB_ENTRY_NOT_EXIST: no need log +// - OB_NEED_RETRY: lock hold by other thread +int ObTransCallbackMgr::get_log_guard(const transaction::ObTxSEQ &write_seq, + ObCallbackListLogGuard &lock_guard, + int &list_idx) +{ + int ret = OB_SUCCESS; + RDLockGuard guard(rwlock_); + list_idx = (write_seq.get_branch() % MAX_CALLBACK_LIST_COUNT); + ObTxCallbackList *list = get_callback_list_(list_idx, true); + if (OB_ISNULL(list)) { + ret = OB_ENTRY_NOT_EXIST; + } else { + int64_t my_epoch = list->get_log_epoch(); + int64_t min_epoch = 0; + int min_epoch_idx =-1; + common::ObByteLock *log_lock = NULL; + if (my_epoch == INT64_MAX) { + ret = OB_ENTRY_NOT_EXIST; + } else if (OB_ISNULL(log_lock = list->try_lock_log())) { + ret = OB_NEED_RETRY; + } else if (!check_list_has_min_epoch_(list_idx, my_epoch, min_epoch, min_epoch_idx)) { + ret = OB_EAGAIN; + ObIMemtable *to_log_memtable = list->get_log_cursor()->get_memtable(); + if (TC_REACH_TIME_INTERVAL(1_s)) { + TRANS_LOG(WARN, "has smaller epoch unlogged", KPC(this), + K(list_idx), K(write_seq), K(my_epoch), K(min_epoch), K(min_epoch_idx), KP(to_log_memtable)); + } + } else { + lock_guard.set(log_lock); + } + if (OB_FAIL(ret) && log_lock) { + log_lock->unlock(); + } + } + return ret; +} + +int ObTransCallbackMgr::fill_log(ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &func) +{ + int ret = OB_SUCCESS; + if (!ATOMIC_LOAD(&callback_lists_) || need_merge_) { + if (OB_UNLIKELY(ctx.list_idx_ > 0)) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(WARN, "list_idx is unexpected", K(ret), K(ctx)); + } else { + ret = fill_from_one_list(ctx, 0, func); + } + } else if (ctx.list_idx_ >= 0) { + ret = fill_from_one_list(ctx, ctx.list_idx_, func); + } else { + ret = fill_from_all_list(ctx, func); + } + return ret; +} + +// +// fill RedoLog from single callback-list +// +// return value: +// - OB_SUCCESS: no remains, all callback is filled +// - OB_ITER_END: stopped due to has smaller write_epoch log hasn't submitted +// - OB_BLOCK_FROZEN: stopped due to memtable of cur callback is waiting previous frozen logged +// - OB_BUF_NOT_ENOUGH: buf can not hold cur callback +int ObTransCallbackMgr::fill_from_one_list(ObTxFillRedoCtx &ctx, + int list_idx, + ObITxFillRedoFunctor &func) +{ + int ret = OB_SUCCESS; + FILL_LOG_TRACE("from one list", K(ctx), K(need_merge_)); + RDLockGuard guard(rwlock_); + int64_t epoch_from = 0, epoch_to = 0; + if (OB_LIKELY(callback_lists_ == NULL) || need_merge_) { + epoch_from = callback_list_.get_log_epoch(); + epoch_to = INT64_MAX; + } else { + calc_list_fill_log_epoch_(list_idx, epoch_from, epoch_to); + } + FILL_LOG_TRACE("start fill list", K(list_idx), K(epoch_from), K(epoch_to), K(ctx)); + if (!need_merge_ && epoch_from == 0) { + ret = OB_ITER_END; // can not fill any callback, because of other list has min write epoch + } else if (epoch_from == INT64_MAX) { + ret = OB_SUCCESS; // no callback to fill + } else { + int16_t idx = -1; + ret = prep_and_fill_from_list_(ctx, func, idx, list_idx, epoch_from, epoch_to); + } + if (OB_SUCC(ret)) { + ctx.is_all_filled_ = true; + } + FILL_LOG_TRACE("fill from one done", K(ctx)); + return ret; +} + +// +// fill redo log from all callback-list +// +// return value: +// - OB_SUCCESS: all callbacks from all callback-list filled +// - OB_EAGAIN: due to parallel logging, must return to flush this list and retry others +// - OB_BLOCK_FROZEN: stopped due to can not logging waiting memtable frozen +// - OB_ITER_END: stopped due to has smaller write_epoch whos log isn't submitted +// - OB_BUF_NOT_ENOUGH: stopped due to buffer can not hold current node +// return policy: +// - if parallel_logging, return if need switch to next list and has +// filled some callback from current list +// - otherwise, either buffer is full or some blocked reason can not fill any more +int ObTransCallbackMgr::fill_from_all_list(ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &func) +{ + int ret = OB_SUCCESS; + FILL_LOG_TRACE("from all list entry", K(ctx)); + RDLockGuard guard(rwlock_); + int list_cnt = get_logging_list_count(); + // record each list's next to fill write_epoch + ObIArray &next_log_epoch_arr = ctx.list_log_epoch_arr_; + if (list_cnt > next_log_epoch_arr.count()) { + if (OB_FAIL(next_log_epoch_arr.reserve(list_cnt))) { + TRANS_LOG(WARN, "reserve space for log epoch fail", K(ret), K(list_cnt)); + } + for (int i=0; OB_SUCC(ret) && i< list_cnt; i++) { + ObTxCallbackList *list = get_callback_list_(i, false); + next_log_epoch_arr.push_back(list->get_log_epoch()); + } + } + // record each list's callback-scope object index in ctx.helper_.callbacks_ array + int16_t callback_scope_idx_arr[list_cnt]; + for (int i =0; i < list_cnt; i++) { + callback_scope_idx_arr[i] = -1; + } + + FILL_LOG_TRACE("start from all list", K(list_cnt), K(ctx)); + + int cur_index = -1; + bool do_return = false; + while (OB_SUCC(ret) && !do_return) { + int index = 0; + int64_t epoch_from = 0, epoch_to = 0; + calc_next_to_fill_log_info_(next_log_epoch_arr, index, epoch_from, epoch_to); + if (index == -1) { + ctx.is_all_filled_ = true; + FILL_LOG_TRACE("all list fill done", K(ctx)); + do_return = true; // all list is totally filled + } else { + int fill_ret = prep_and_fill_from_list_(ctx, + func, + callback_scope_idx_arr[index], + index, + epoch_from, + epoch_to); + FILL_LOG_TRACE("one fill round 1/2", K(fill_ret), K(index), K(epoch_from), K(epoch_to), K(ctx)); + bool try_other_lists = false; + if (OB_SUCCESS == fill_ret) { + // cur list is all filled + next_log_epoch_arr.at(index) = INT64_MAX; + // update epoch to point next, thus can consume from other list + if (epoch_to == INT64_MAX) { + // all filled + ctx.is_all_filled_ = true; + ret = fill_ret; + do_return = true; + } else { + ctx.cur_epoch_ = epoch_to; + try_other_lists = true; + } + } else if (OB_BUF_NOT_ENOUGH == fill_ret) { + // buffer is full, must return to flush + next_log_epoch_arr.at(index) = ctx.cur_epoch_; + ret = fill_ret; + do_return = true; + } else if (OB_BLOCK_FROZEN == fill_ret) { + // blocked, maybe has fill some data, maybe none + next_log_epoch_arr.at(index) = ctx.cur_epoch_; + try_other_lists = true; + } else if (OB_ITER_END == fill_ret) { + // this list has remains, but epoch larger than min epoch of other lists + next_log_epoch_arr.at(index) = ctx.next_epoch_; + // update epoch to point next, thus can consume from other list + ctx.cur_epoch_ = epoch_to; + try_other_lists = true; + } else { + TRANS_LOG(WARN, "fill redo from list fail", K(fill_ret), K(index)); + ret = fill_ret; + } + // when parallel logging, seperate log-entry for each callback-list + if (!do_return && is_parallel_logging_() && ctx.not_empty()) { + try_other_lists = false; + ret = fill_ret; + if (OB_SUCCESS == fill_ret) { + ret = OB_EAGAIN; + } + do_return = true; + } + + int choosen_list_fill_ret = fill_ret; + // fill from other lists, this can be in two situations: + // 1. parallel logging, but the first list can not fill any data + // 2. serial logging, and buf is not full, need fill from others + FILL_LOG_TRACE("one fill round 2/2", K(fill_ret), K(try_other_lists), K(ctx)); + if (try_other_lists && (list_cnt == 1 || ctx.cur_epoch_ != epoch_to)) { + ret = fill_ret; + ctx.is_all_filled_ = (list_cnt == 1) && (OB_SUCCESS == fill_ret); + do_return = true; + } else if (try_other_lists) { + const int save_fill_count = ctx.fill_count_; + // can only consume this epoch + int64_t fill_epoch = ctx.cur_epoch_; + bool all_others_reach_tail = true; + int last_fail = OB_SUCCESS; + for (int i = 0; i < list_cnt && !do_return; i++) { + if (i == index) { + } else if (next_log_epoch_arr.at(i) == INT64_MAX) { + // nothing to fill, skip it + } else if (next_log_epoch_arr.at(i) == fill_epoch) { + FILL_LOG_TRACE("start fill others >>", K(i), K(fill_epoch), K(ctx)); + fill_ret = prep_and_fill_from_list_(ctx, + func, + callback_scope_idx_arr[i], + i, + fill_epoch, + fill_epoch); + FILL_LOG_TRACE("fill others done <<", K(fill_ret), K(ctx)); + if (OB_SUCCESS == fill_ret) { + // this list is fully filled, continue to fill from others + next_log_epoch_arr.at(i) = INT64_MAX; + } else if (FALSE_IT(all_others_reach_tail = false)) { + // not reach tail + } else if (OB_ITER_END == fill_ret) { + // this list is filled with this epoch, but its has remains + next_log_epoch_arr.at(i) = ctx.next_epoch_; + } else if (FALSE_IT(last_fail = fill_ret)) { + // failure occurs, either retryable or fatal + } else if (OB_BUF_NOT_ENOUGH == fill_ret) { + next_log_epoch_arr.at(i) = ctx.cur_epoch_; + ret = fill_ret; + do_return = true; + } else if (OB_BLOCK_FROZEN == fill_ret) { + // blocked, try others + next_log_epoch_arr.at(i) = ctx.cur_epoch_; + } else { + // other error, give up + TRANS_LOG(WARN, "fill redo from callback-list fail", K(ret), K(i)); + next_log_epoch_arr.at(i) = ctx.cur_epoch_; + ret = fill_ret; + do_return = true; + } + if (!do_return && is_parallel_logging_() && ctx.not_empty()) { + ret = fill_ret; + // when parallel logging, seprate log-entry for each callback-list + if (OB_SUCCESS == fill_ret) { + ret = OB_EAGAIN; // return OB_EAGAIN indicate other list has remains + } + do_return = true; + } + } else { + // this list is skipped + all_others_reach_tail = false; + } + } + if (!do_return) { + if (OB_SUCC(ret) && (OB_SUCCESS == choosen_list_fill_ret) && all_others_reach_tail) { + // all list reach tail, no need next round + ctx.is_all_filled_ = true; + do_return = true; + FILL_LOG_TRACE("all list filled, remians 0, return now", K(ctx)); + } else if (ctx.fill_count_ - save_fill_count == 0) { + // no extra filled from other list, no need next round + ob_assert(last_fail != OB_SUCCESS); + // if first list is filled without error, should return last fill error + if (choosen_list_fill_ret == OB_SUCCESS || choosen_list_fill_ret == OB_ITER_END) { + ret = last_fail; + } else { + ret = choosen_list_fill_ret; + } + do_return = true; + } else { + // go ahead, next round + } + } + } + FILL_LOG_TRACE("one round is done", K(do_return), K(ctx)); + } + } + if (!ctx.is_all_filled_) { + ob_assert(ret != OB_SUCCESS); + } + FILL_LOG_TRACE("done fill from all list", K(list_cnt), K(ctx)); + return ret; +} + +inline bool check_dup_tablet_(ObITransCallback *callback_ptr) +{ + bool is_dup_tablet = false; + int64_t tmp_ret = OB_SUCCESS; + + // If id is a dup table tablet => true + // If id is not a dup table tablet => false + if (MutatorType::MUTATOR_ROW == callback_ptr->get_mutator_type()) { + const ObMvccRowCallback *row_iter = static_cast(callback_ptr); + const ObTabletID &target_tablet = row_iter->get_tablet_id(); + // if (OB_TMP_FAIL(mem_ctx_->get_trans_ctx()->merge_tablet_modify_record_(target_tablet))) { + // TRANS_LOG_RET(WARN, tmp_ret, "merge tablet modify record failed", K(tmp_ret), + // K(target_tablet), KPC(row_iter)); + // } + // check dup table + } + + return is_dup_tablet; +} + +int ObTransCallbackMgr::log_submitted(const ObCallbackScopeArray &callbacks, share::SCN scn, int &submitted) +{ + int ret = OB_SUCCESS; + ObIMemtable *last_mt = NULL; + ARRAY_FOREACH(callbacks, i) { + ObCallbackScope scope = callbacks.at(i); + if (!scope.is_empty()) { + int cnt = 0; + ObITransCallbackIterator cursor = scope.start_; + do { + ObITransCallback *iter = *cursor; + OB_ASSERT(iter->need_submit_log()); + if (OB_FAIL(iter->log_submitted_cb(scn, last_mt))) { + TRANS_LOG(ERROR, "fail to log_submitted cb", K(ret), KPC(iter)); + ob_abort(); + } // check dup table tx + else if(check_dup_tablet_(iter)) { + // mem_ctx_->get_trans_ctx()->set_dup_table_tx_(); + } + ++cnt; + ++submitted; + } while (OB_SUCC(ret) && cursor++ != scope.end_); + OB_ASSERT(cnt == scope.cnt_); + if (OB_SUCC(ret)) { + // update log cursor + ret = scope.host_->submit_log_succ(callbacks.at(i)); + } + } + } + return ret; +} + +int ObTransCallbackMgr::log_sync_succ(const ObCallbackScopeArray &callbacks, + const share::SCN scn, + int64_t &sync_cnt) +{ + int ret = OB_SUCCESS; + sync_cnt = 0; + const bool serial_final = is_serial_final_(); + if (!serial_final) { + update_serial_sync_scn_(scn); + } + ARRAY_FOREACH(callbacks, i) { + const ObCallbackScope &scope = callbacks.at(i); + if (!scope.is_empty()) { + if (OB_FAIL(scope.host_->sync_log_succ(scn, scope.cnt_))) { + TRANS_LOG(ERROR, "sync succ fail", K(ret)); + } else { + sync_cnt += scope.cnt_; + } + } else { + ob_abort(); + } + } + return ret; +} + +int ObTransCallbackMgr::log_sync_fail(const ObCallbackScopeArray &callbacks, + const share::SCN scn, int64_t &removed_cnt) { int ret = OB_SUCCESS; removed_cnt = 0; - - // TODO(handora.qc): remove it in the future - RDLockGuard guard(rwlock_); - - if (callbacks.is_empty()) { - // pass empty callbacks - } else if (OB_FAIL(callback_list_.sync_log_fail(callbacks, removed_cnt))) { - TRANS_LOG(ERROR, "sync log fail", K(ret)); + ARRAY_FOREACH(callbacks, i) { + const ObCallbackScope &scope = callbacks.at(i); + int64_t rm_cnt = 0; + if (!scope.is_empty()) { + if (OB_FAIL(scope.host_->sync_log_fail(scope, scn, rm_cnt))) { + TRANS_LOG(ERROR, "calblack fail", K(ret)); + } else { + OB_ASSERT(rm_cnt == scope.cnt_); + removed_cnt += rm_cnt; + } + } } - return ret; } -void ObTransCallbackMgr::update_checksum(const uint64_t checksum, - const SCN checksum_scn) +// when recover from checkpoint, update checksum info for CallbackList +int ObTransCallbackMgr::update_checksum(const ObIArray &checksum, + const ObIArray &checksum_scn) { - callback_list_.update_checksum(checksum, checksum_scn); + int ret = OB_SUCCESS; + // extend callback list if need, this can only happened since 4.3 + if (checksum.count() > 1) { + OB_ASSERT(checksum.count() == MAX_CALLBACK_LIST_COUNT); + if (OB_ISNULL(callback_lists_) && + OB_FAIL(extend_callback_lists_(MAX_CALLBACK_LIST_COUNT - 1))) { + TRANS_LOG(WARN, "expand calblack_lists failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + CALLBACK_LISTS_FOREACH(idx, list) { + list->update_checksum(checksum.at(idx), checksum_scn.at(idx)); + } + } + return ret; } int64_t ObTransCallbackMgr::inc_pending_log_size(const int64_t size) { int64_t new_size = -1; - if (!for_replay_) { + if (!for_replay_ && !is_parallel_logging_()) { int64_t old_size = ATOMIC_FAA(&pending_log_size_, size); new_size = ATOMIC_LOAD(&pending_log_size_); if (old_size < 0 || new_size < 0) { @@ -554,7 +1204,7 @@ int64_t ObTransCallbackMgr::inc_pending_log_size(const int64_t size) void ObTransCallbackMgr::try_merge_multi_callback_lists(const int64_t new_size, const int64_t size, const bool is_logging_blocked) { - if (!for_replay_) { + if (OB_UNLIKELY(need_merge_) && !for_replay_) { int64_t old_size = new_size - size; if (size < 0 || new_size < 0 || old_size < 0) { } else if ((0 != GCONF._private_buffer_size @@ -570,62 +1220,81 @@ void ObTransCallbackMgr::try_merge_multi_callback_lists(const int64_t new_size, int ObTransCallbackMgr::get_memtable_key_arr(ObMemtableKeyArray &memtable_key_arr) { int ret = OB_SUCCESS; - - if (OB_FAIL(callback_list_.get_memtable_key_arr_w_timeout(memtable_key_arr))) { - if (OB_ITER_STOP == ret) { - ret = OB_SUCCESS; - } else { - TRANS_LOG(WARN, "lifo callback get memtablekey fail", K(ret), K(memtable_key_arr)); + int fail_at = 0; + if (need_merge_) { // OLD (before 4.3) + ret = callback_list_.get_memtable_key_arr_w_timeout(memtable_key_arr); + if (OB_ITER_STOP == ret) { ret = OB_SUCCESS; } + } else { // NEW (since 4.3) + CALLBACK_LISTS_FOREACH(idx, list) { + fail_at = idx; + ret = list->get_memtable_key_arr_w_timeout(memtable_key_arr); + if (OB_ITER_STOP == ret) { ret = OB_SUCCESS; } } - } else { - //do nothing } - + if (OB_FAIL(ret)) { + TRANS_LOG(WARN, "get memtablekey fail", K(ret), K(fail_at), K(memtable_key_arr)); + } return ret; } -void ObTransCallbackMgr::acquire_callback_list() +int ObTransCallbackMgr::acquire_callback_list(const bool new_epoch, const bool need_merge) { int64_t stat = ATOMIC_LOAD(¶llel_stat_); int64_t tid = get_itid() + 1; - if (0 == stat) { + if (0 == stat) { // first thread, no parallel if (!ATOMIC_BCAS(¶llel_stat_, 0, tid << 32)) { ATOMIC_STORE(¶llel_stat_, PARALLEL_STMT); } - } else if (tid == (stat >> 32)) { + } else if (tid == (stat >> 32)) { // same thread nested, no parallel if (!ATOMIC_BCAS(¶llel_stat_, stat, stat + 1)) { TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "Unexpected status", K(this), K(tid_), K(ref_cnt_), K(tid)); } - } else { + } else { // has parallel // ATOMIC_STORE(¶llel_stat_, PARALLEL_STMT); } + // mark callback_list need merge into main + // this is compatible with version before 4.3 + if (ATOMIC_LOAD(&need_merge_) != need_merge) { + ATOMIC_STORE(&need_merge_, need_merge); + } + int slot = 0; + // inc write_epoch + // for each write epoch the first thread always stay in slot 0 + // other thread will stay in slot by its offset with first thread + if (new_epoch) { + ++write_epoch_; + write_epoch_start_tid_ = tid; + slot = 0; + } else if (tid == write_epoch_start_tid_) { + slot = 0; + } else { + // to ensure slot is positive: (m + (a - b) % m) % m + slot = (MAX_CALLBACK_LIST_COUNT + ((tid - write_epoch_start_tid_) % MAX_CALLBACK_LIST_COUNT)) % MAX_CALLBACK_LIST_COUNT; + } + return slot; } void ObTransCallbackMgr::revert_callback_list() { int64_t stat = ATOMIC_LOAD(¶llel_stat_); + bool need_merge = ATOMIC_LOAD(&need_merge_); const int64_t tid = get_itid() + 1; - const int64_t slot = tid % MAX_CALLBACK_LIST_COUNT; - int64_t cnt = 0; - if (0 == stat) { - WRLockGuard guard(rwlock_); - // - if (OB_NOT_NULL(callback_lists_)) { - cnt = callback_list_.concat_callbacks(callback_lists_[slot]); - add_slave_list_merge_cnt(cnt); - } - } else if (tid == (stat >> 32)) { + const int slot = tid % MAX_CALLBACK_LIST_COUNT; + // if no parallel til now, all callbacks in main list, no need merge + if (tid == (stat >> 32)) { if (0 == ref_cnt_) { UNUSED(ATOMIC_BCAS(¶llel_stat_, stat, 0)); } else { UNUSED(ATOMIC_BCAS(¶llel_stat_, stat, stat - 1)); } - } else { - // We need merge callback list for causality bwteen them + need_merge = false; + } + // compatible with before version 4.3, merge to main list + if (need_merge) { WRLockGuard guard(rwlock_); if (OB_NOT_NULL(callback_lists_)) { - cnt = callback_list_.concat_callbacks(callback_lists_[slot]); + int64_t cnt = callback_list_.concat_callbacks(callback_lists_[slot]); add_slave_list_merge_cnt(cnt); } } @@ -649,29 +1318,74 @@ void ObTransCallbackMgr::set_for_replay(const bool for_replay) } } -int ObTransCallbackMgr::replay_fail(const SCN scn) +void ObTransCallbackMgr::replay_begin(const bool parallel_replay, share::SCN scn) { - return callback_list_.replay_fail(scn); + UNUSED(scn); + parallel_replay_ = parallel_replay; } -int ObTransCallbackMgr::replay_succ(const SCN scn) +int ObTransCallbackMgr::replay_fail(const int16_t callback_list_idx, const SCN scn) { - return OB_SUCCESS; + int ret = OB_SUCCESS; + // if not reach serial replay final, replay maybe in multiple list + // must try do rollback on all list + if (callback_list_idx == 0 && !is_serial_final_()) { + CALLBACK_LISTS_FOREACH(idx, list) { + ret = list->replay_fail(scn, true/*is serial replayed scn*/); + } + } else { + ObTxCallbackList *list = get_callback_list_(callback_list_idx, true); + if (OB_ISNULL(list)) { + // callback_lists is not extended due to replay row is skipped + } else { + ret = list->replay_fail(scn, false/*is serial replay scn*/); + } + } + return ret; +} + +int ObTransCallbackMgr::replay_succ(const int16_t callback_list_idx, const SCN scn) +{ + // when replay succ, update sync_scn + int ret = OB_SUCCESS; + if (callback_list_idx == 0 && !is_serial_final_()) { + // it's replaying log in tx-log queue, involve multiple callback-list maybe + update_serial_sync_scn_(scn); + } else { + ObTxCallbackList *list = get_callback_list_(callback_list_idx, true); + if (OB_ISNULL(list)) { + // callback_lists is not extended due to replay row is skipped + } else { + ret = list->replay_succ(scn); + } + } + return ret; } int ObTransCallbackMgr::trans_end(const bool commit) { int ret = common::OB_SUCCESS; - // If the txn ends abnormally, there may still be tasks in execution. Our - // solution is that before the txn resets, all callback_lists need be - // cleaned up after blocking new writes (through end_code). So if PDML - // exists and some data is cached in callback_lists, we need merge them into - // main callback_list - merge_multi_callback_lists(); - if (commit) { - ret = callback_list_.tx_commit(); - } else { - ret = callback_list_.tx_abort(); + // abort transaction, skip the checksum cacluation + // which also skip remove callback order contraint checks + if (!commit) { + set_skip_checksum_calc(); + } + if (OB_UNLIKELY(ATOMIC_LOAD(&need_merge_))) { // OLD (before 4.3) + // If the txn ends abnormally, there may still be tasks in execution. Our + // solution is that before the txn resets, all callback_lists need be + // cleaned up after blocking new writes (through end_code). So if PDML + // exists and some data is cached in callback_lists, we need merge them into + // main callback_list + merge_multi_callback_lists(); + ret = commit ? callback_list_.tx_commit() : callback_list_.tx_abort(); + } else { // New (since 4.3) + if (OB_LIKELY(ATOMIC_LOAD(&callback_lists_) == NULL)) { + ret = commit ? callback_list_.tx_commit() : callback_list_.tx_abort(); + } else { + CALLBACK_LISTS_FOREACH(idx, list) { + ret = commit ? list->tx_commit() : list->tx_abort(); + } + } } if (OB_SUCC(ret)) { wakeup_waiting_txns_(); @@ -679,19 +1393,65 @@ int ObTransCallbackMgr::trans_end(const bool commit) return ret; } -void ObTransCallbackMgr::calc_checksum_all() +int ObTransCallbackMgr::calc_checksum_all(ObIArray &checksum) { - callback_list_.tx_calc_checksum_all(); + RDLockGuard guard(rwlock_); + int ret = OB_SUCCESS; + if (OB_UNLIKELY(ATOMIC_LOAD(&need_merge_)) || OB_LIKELY(callback_lists_ == NULL)) { + callback_list_.tx_calc_checksum_all(); + ret = checksum.push_back(callback_list_.get_checksum()); + } else { + CALLBACK_LISTS_FOREACH(idx, list) { + list->tx_calc_checksum_all(); + ret = checksum.push_back(list->get_checksum()); + }; + } + return ret; } void ObTransCallbackMgr::print_callbacks() { - callback_list_.tx_print_callback(); + RDLockGuard guard(rwlock_); + if (need_merge_) { + callback_list_.tx_print_callback(); + } else { + int ret = OB_SUCCESS; + CALLBACK_LISTS_FOREACH(idx, list) { + _TRANS_LOG(INFO, "print callback at CallbackList[%d]:", idx); + list->tx_print_callback(); + } + } +} + +int ObTransCallbackMgr::get_callback_list_stat(ObIArray &stats) +{ + RDLockGuard guard(rwlock_); + int ret = OB_SUCCESS; + if (rwlock_.try_rdlock()) { + if (need_merge_) { + if (OB_SUCC(stats.prepare_allocate(1))) { + ret = callback_list_.get_stat_for_display(stats.at(0)); + } + } else if (OB_SUCC(stats.prepare_allocate(get_callback_list_count()))) { + CALLBACK_LISTS_FOREACH(idx, list) { + ret = list->get_stat_for_display(stats.at(idx)); + } + } + rwlock_.unlock(); + } + return ret; } void ObTransCallbackMgr::elr_trans_preparing() { - callback_list_.tx_elr_preparing(); + if (ATOMIC_LOAD(&need_merge_)) { + callback_list_.tx_elr_preparing(); + } else { + int ret = OB_SUCCESS; + CALLBACK_LISTS_FOREACH(idx, list) { + list->tx_elr_preparing(); + } + } } void ObTransCallbackMgr::trans_start() @@ -708,7 +1468,6 @@ int ObMvccRowCallback::before_append(const bool is_replay) TRANS_LOG(ERROR, "memtable is NULL", K(ret)); } else if (!is_replay) { inc_unsubmitted_cnt_(); - inc_unsynced_cnt_(); } return ret; @@ -719,11 +1478,18 @@ void ObMvccRowCallback::after_append(const bool is_replay) // do nothing } -int ObMvccRowCallback::log_submitted() +int ObMvccRowCallback::log_submitted(const SCN scn, ObIMemtable *&last_mt) { int ret = OB_SUCCESS; - if (OB_NOT_NULL(memtable_)) { + if (scn.is_valid()) { + if (last_mt != memtable_) { + memtable_->set_rec_scn(scn); + memtable_->set_max_end_scn(scn); + last_mt = memtable_; + } + (void)tnode_->fill_scn(scn); + } if (OB_FAIL(dec_unsubmitted_cnt_())) { TRANS_LOG(ERROR, "dec unsubmitted cnt failed", K(ret), K(*this)); } @@ -731,21 +1497,6 @@ int ObMvccRowCallback::log_submitted() ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "memtable is NULL", K(ret)); } - - return ret; -} - -int ObMvccRowCallback::undo_log_submitted() -{ - int ret = OB_SUCCESS; - - if (OB_ISNULL(memtable_)) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "memtable is NULL", K(ret)); - } else { - inc_unsubmitted_cnt_(); - } - return ret; } @@ -757,8 +1508,8 @@ bool ObMvccRowCallback::is_logging_blocked() const ObTransID trans_id; if (OB_FAIL(get_trans_id(trans_id))) { TRANS_LOG(WARN, "fail to get trans_id", K(ret)); - } else { - TRANS_LOG(WARN, "block logging", K(is_blocked), KP(memtable_), + } else if (REACH_TIME_INTERVAL(1000000)) { + TRANS_LOG(WARN, "block logging", K_(epoch), K(is_blocked), KP(memtable_), K(memtable_->get_key().get_tablet_id()), K(trans_id)); } } @@ -780,13 +1531,10 @@ int ObMvccRowCallback::del() old_row_.data_ = NULL; } - if (need_submit_log_ && need_fill_redo_) { - log_submitted(); + if (need_submit_log_) { + ObIMemtable *last_mt = NULL; + log_submitted(share::SCN(), last_mt); } - if (need_fill_redo_) { - dec_unsynced_cnt_(); - } - // set block_frozen_memtable if the first callback is linked to a logging_blocked memtable // to prevent the case where the first callback is removed but the block_frozen_memtable pointer is still existed // clear block_frozen_memtable once a callback is deleted @@ -948,7 +1696,7 @@ ObTransCtx *ObMvccRowCallback::get_trans_ctx() const } int ObMvccRowCallback::calc_checksum(const SCN checksum_scn, - ObBatchChecksum *checksumer) + TxChecksum *checksumer) { ObRowLatchGuard guard(value_.latch_); @@ -962,6 +1710,8 @@ int ObMvccRowCallback::calc_checksum(const SCN checksum_scn, } else if (checksum_scn <= scn_) { tnode_->checksum(*checksumer); ((ObMemtableDataHeader *)tnode_->buf_)->checksum(*checksumer); + checksumer->cnt_++; + checksumer->scn_ = scn_; } } @@ -974,9 +1724,9 @@ int ObMvccRowCallback::checkpoint_callback() ObRowLatchGuard guard(value_.latch_); - if (need_submit_log_ || need_fill_redo_) { + if (need_submit_log_) { ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "checkpoint never called on unsynced callback", KPC(this)); + TRANS_LOG(ERROR, "checkpoint never called on submitted callback", KPC(this)); } else if (OB_FAIL(value_.remove_callback(*this))) { TRANS_LOG(ERROR, "remove callback from trans node failed", K(ret), K(*this)); } @@ -1151,9 +1901,7 @@ int ObMvccRowCallback::rollback_callback() unlink_trans_node(); } - if (need_submit_log_ - && need_fill_redo_ - && SCN::max_scn() == scn_) { + if (need_submit_log_ && SCN::max_scn() == scn_) { ctx_.inc_pending_log_size(-1 * data_size_); } @@ -1260,46 +2008,21 @@ int64_t ObMvccRowCallback::to_string(char *buf, const int64_t buf_len) const { int64_t pos = 0; databuff_printf(buf, buf_len, pos, - "[this=%p, ctx=%s, is_link=%d, need_fill_redo=%d, " + "[this=%p, ctx=%s, is_link=%d, need_submit_log=%d, " "value=%s, tnode=(%s), " "seq_no=%ld, memtable=%p, scn=%s", - this, to_cstring(ctx_), is_link_, need_fill_redo_, + this, to_cstring(ctx_), is_link_, need_submit_log_, to_cstring(value_), NULL == tnode_ ? "null" : to_cstring(*tnode_), seq_no_.cast_to_int(), memtable_, to_cstring(scn_)); return pos; } -int ObMvccRowCallback::log_sync(const SCN scn) +int ObMvccRowCallback::log_sync_fail(const share::SCN max_committed_scn) { int ret = OB_SUCCESS; - - memtable_->set_rec_scn(scn); - memtable_->set_max_end_scn(scn); - (void)tnode_->fill_scn(scn); - ctx_.update_max_submitted_seq_no(seq_no_); - if (OB_FAIL(dec_unsynced_cnt_())) { - TRANS_LOG(ERROR, "memtable dec unsynced cnt error", K(ret), K(scn), - K(memtable_->get_unsynced_cnt())); - } else { - // do nothing - } - - return ret; -} - -int ObMvccRowCallback::log_sync_fail() -{ - int ret = OB_SUCCESS; - ObRowLatchGuard guard(value_.latch_); - - if (OB_FAIL(dec_unsynced_cnt_())) { - TRANS_LOG(ERROR, "memtable dec unsynced cnt error", K(ret), - K(memtable_->get_unsynced_cnt())); - } else { - unlink_trans_node(); - } - + unlink_trans_node(); + memtable_->set_max_end_scn(max_committed_scn, true); return ret; } @@ -1310,16 +2033,10 @@ int ObMvccRowCallback::clean_unlog_cb() // or gracefully) will ensure the invokation of all callbacks(whether succeed // or fail). So we add defensive code here for safety. - if (need_fill_redo_ && !need_submit_log_) { - TRANS_LOG(ERROR, "all callbacks must be invoked before leader switch", K(*this)); - } else if (!need_fill_redo_ && need_submit_log_) { - TRANS_LOG(ERROR, "It will never on success before submit log", K(*this)); - } else if (need_fill_redo_ && need_submit_log_) { + if (need_submit_log_) { unlink_trans_node(); need_submit_log_ = false; - need_fill_redo_ = false; dec_unsubmitted_cnt_(); - dec_unsynced_cnt_(); } return ret; } @@ -1332,14 +2049,6 @@ void ObMvccRowCallback::inc_unsubmitted_cnt_() } } -void ObMvccRowCallback::inc_unsynced_cnt_() -{ - if (OB_NOT_NULL(memtable_)) { - memtable_->inc_unsynced_cnt(); - ctx_.inc_unsynced_cnt(); - } -} - int ObMvccRowCallback::dec_unsubmitted_cnt_() { int ret = OB_SUCCESS; @@ -1352,23 +2061,135 @@ int ObMvccRowCallback::dec_unsubmitted_cnt_() return ret; } -int ObMvccRowCallback::dec_unsynced_cnt_() -{ - int ret = OB_SUCCESS; - - if (OB_NOT_NULL(memtable_)) { - ret = memtable_->dec_unsynced_cnt(); - ctx_.dec_unsynced_cnt(); - } - - return ret; -} - MutatorType ObITransCallback::get_mutator_type() const { return MutatorType::MUTATOR_ROW; } +void ObTransCallbackMgr::print_statistics(char *buf, const int64_t buf_len, int64_t &pos) const +{ + common::databuff_printf(buf, buf_len, pos, + "callback_list:{" + "cnt=%d need_merge=%d " + "stat:[", + get_callback_list_count(), + need_merge_); + if (need_merge_) { + common::databuff_printf(buf, buf_len, pos, + "main=%ld, slave=%ld, merge=%ld, ", + get_callback_main_list_append_count(), + get_callback_slave_list_append_count(), + get_callback_slave_list_merge_count()); + } + common::databuff_printf(buf, buf_len, pos, + "tx_end=%ld, rollback_to=%ld, " + "fast_commit=%ld, remove_memtable=%ld]", + get_callback_remove_for_trans_end_count(), + get_callback_remove_for_rollback_to_count(), + get_callback_remove_for_fast_commit_count(), + get_callback_remove_for_remove_memtable_count()); + if (!need_merge_) { + common::databuff_printf(buf, buf_len, pos, + " detail:[(log_epoch,length,logged,synced,appended,removed,unlog_removed,branch_removed)|"); + int ret = OB_SUCCESS; + CALLBACK_LISTS_FOREACH_CONST(idx, list) { + int64_t a = list->get_length(), + b = list->get_logged(), + c = list->get_synced(), + d = list->get_appended(), + e = list->get_removed(), + f = list->get_unlog_removed(), + g = list->get_branch_removed(); + if (a || b || c || d || e || f || g) { + int64_t log_epoch = list->get_log_epoch(); + log_epoch = log_epoch == INT64_MAX ? -1 : log_epoch; + common::databuff_printf(buf, buf_len, pos, "%d:(%ld,%ld,%ld,%ld,%ld,%ld,%ld,%ld)|", idx, log_epoch, a, b, c, d, e, f, g); + } + } + common::databuff_printf(buf, buf_len, pos, "]}"); + } +} + +bool ObTransCallbackMgr::find(ObITxCallbackFinder &func) +{ + bool found = false; + int ret = OB_SUCCESS; + CALLBACK_LISTS_FOREACH(idx, list) { + if (list->find(func)) { + found = true; + ret = OB_ITER_END; + } + } + return found; +} + +inline ObTxCallbackList *ObTransCallbackMgr::get_callback_list_(const int16_t index, const bool nullable) +{ + if (index == 0) { + return &callback_list_; + } + if (callback_lists_) { + OB_ASSERT(index < MAX_CALLBACK_LIST_COUNT); + return &callback_lists_[index - 1]; + } else if (!nullable) { + ob_abort(); + } + return NULL; +} + +void ObTransCallbackMgr::check_all_redo_flushed() +{ + bool ok = true; + int ret = OB_SUCCESS; + CALLBACK_LISTS_FOREACH(idx, list) { + ok &= list->check_all_redo_flushed(false/*quite*/); + } + if (!ok) { + usleep(1000000); + ob_abort(); + } +} +__attribute__((noinline)) +int ObTransCallbackMgr::get_logging_list_count() const +{ + return (!need_merge_ && callback_lists_) ? MAX_CALLBACK_LIST_COUNT : 1; +} + +bool ObTransCallbackMgr::pending_log_size_too_large(const transaction::ObTxSEQ &write_seq_no, + const int64_t limit) +{ + if (is_parallel_logging_()) { + ObTxCallbackList *list = get_callback_list_(write_seq_no.get_branch() % MAX_CALLBACK_LIST_COUNT, true); + return list ? list->pending_log_too_large(limit) : 0; + } else { + return ATOMIC_LOAD(&pending_log_size_) > limit; + } +} + +void ObTransCallbackMgr::set_parallel_logging(const share::SCN serial_final_scn) +{ + serial_final_scn_.atomic_set(serial_final_scn); +} + +void ObTransCallbackMgr::update_serial_sync_scn_(const share::SCN scn) +{ + // push all callback list's sync scn up to at least serial final scn + // transform to append only replay mode + serial_sync_scn_.atomic_store(scn); + if (serial_sync_scn_ == serial_final_scn_) { + RDLockGuard guard(rwlock_); + int ret = OB_SUCCESS; + CALLBACK_LISTS_FOREACH(idx, list) { + list->inc_update_sync_scn(scn); + } + } +} + +void ObTransCallbackMgr::set_skip_checksum_calc() +{ + ATOMIC_STORE(&skip_checksum_, true); +} + }; // end namespace mvcc }; // end namespace oceanbase diff --git a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h index 56591633f8..8a95c6e0a7 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h +++ b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h @@ -36,8 +36,12 @@ class ObMemtableCtxCbAllocator; class ObIMemtable; class ObMemtable; class ObCallbackScope; +typedef ObIArray ObCallbackScopeArray; enum class MutatorType; - +class ObTxFillRedoCtx; +class RedoLogEpoch; +class ObCallbackListLogGuard; +class ObTxCallbackListStat; class ObITransCallback; struct RedoDataNode { @@ -102,8 +106,8 @@ public: } ObITransCallback* operator*() { return cur_; } ObITransCallback* operator*() const { return cur_; } - bool operator==(const ObITransCallbackIterator &that) { return cur_ == that.cur_; } - bool operator!=(const ObITransCallbackIterator &that) { return cur_ != that.cur_; } + bool operator==(const ObITransCallbackIterator &that) const { return cur_ == that.cur_; } + bool operator!=(const ObITransCallbackIterator &that) const { return cur_ != that.cur_; } ObITransCallbackIterator operator+(int i) { ObITransCallbackIterator ret(cur_); @@ -134,13 +138,15 @@ public: } ObITransCallbackIterator operator++(int) // iter++ { + ObITransCallback *cur_save = cur_; cur_ = cur_->get_next(); - return ObITransCallbackIterator(cur_->get_prev()); + return ObITransCallbackIterator(cur_save); } ObITransCallbackIterator operator--(int) // iter-- { + ObITransCallback *cur_save = cur_; cur_ = cur_->get_prev(); - return ObITransCallbackIterator(cur_->get_next()); + return ObITransCallbackIterator(cur_save); } private: ObITransCallback *cur_; @@ -174,7 +180,7 @@ public: }; friend class ObITransCallbackIterator; - enum { MAX_CALLBACK_LIST_COUNT = OB_MAX_CPU_NUM }; + enum { MAX_CALLBACK_LIST_COUNT = transaction::MAX_CALLBACK_LIST_COUNT }; enum { MAX_CB_ALLOCATOR_COUNT = OB_MAX_CPU_NUM }; enum { PARALLEL_STMT = -1 @@ -182,11 +188,17 @@ public: public: ObTransCallbackMgr(ObIMvccCtx &host, ObMemtableCtxCbAllocator &cb_allocator) : host_(host), - callback_list_(*this), + skip_checksum_(false), + callback_list_(*this, 0), callback_lists_(NULL), rwlock_(ObLatchIds::MEMTABLE_CALLBACK_LIST_MGR_LOCK), parallel_stat_(0), + write_epoch_(0), + write_epoch_start_tid_(0), + need_merge_(false), for_replay_(false), + serial_final_scn_(share::SCN::max_scn()), + serial_sync_scn_(share::SCN::min_scn()), callback_main_list_append_count_(0), callback_slave_list_append_count_(0), callback_slave_list_merge_count_(0), @@ -209,66 +221,65 @@ public: void before_append(ObITransCallback *node); void after_append(ObITransCallback *node, const int ret_code); void trans_start(); - void calc_checksum_all(); + int calc_checksum_all(ObIArray &checksum); void print_callbacks(); + int get_callback_list_stat(ObIArray &stats); void elr_trans_preparing(); int trans_end(const bool commit); - int replay_fail(const share::SCN scn); - int replay_succ(const share::SCN scn); + void replay_begin(const bool parallel_replay, share::SCN ccn); +public: + int replay_fail(const int16_t callback_list_idx, const share::SCN scn); + int replay_succ(const int16_t callback_list_idx, const share::SCN scn); int rollback_to(const transaction::ObTxSEQ seq_no, - const transaction::ObTxSEQ from_seq_no); + const transaction::ObTxSEQ from_seq_no, + const share::SCN replay_scn); void set_for_replay(const bool for_replay); bool is_for_replay() const { return ATOMIC_LOAD(&for_replay_); } - int remove_callbacks_for_fast_commit(const ObITransCallback *generate_cursor, - bool &meet_generate_cursor); - int remove_callback_for_uncommited_txn( - const memtable::ObMemtableSet *memtable_set, - const share::SCN max_applied_scn); + int remove_callbacks_for_fast_commit(const int16_t callback_list_idx, const share::SCN stop_scn); + int remove_callbacks_for_fast_commit(const ObCallbackScopeArray &callbacks_arr); + int remove_callback_for_uncommited_txn(const memtable::ObMemtableSet *memtable_set); int get_memtable_key_arr(transaction::ObMemtableKeyArray &memtable_key_arr); - void acquire_callback_list(); + int acquire_callback_list(const bool new_epoch, const bool need_merge); void revert_callback_list(); - // TODO by fengshuo.fs: fix this implement - ObITransCallbackIterator begin() { return ObITransCallbackIterator(get_guard_()); } - ObITransCallbackIterator end() { return ObITransCallbackIterator(get_guard_()); } + int get_tx_seq_replay_idx(const transaction::ObTxSEQ seq) const; common::SpinRWLock& get_rwlock() { return rwlock_; } private: void wakeup_waiting_txns_(); + int extend_callback_lists_(const int16_t cnt); public: - int sync_log_fail(const ObCallbackScope &callbacks, - int64_t &removed_cnt); + int fill_log(ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &func); + int log_submitted(const ObCallbackScopeArray &callbacks, share::SCN scn, int &submitted); + int log_sync_succ(const ObCallbackScopeArray &callbacks, const share::SCN scn, int64_t &sync_cnt); + int log_sync_fail(const ObCallbackScopeArray &callbacks, const share::SCN scn, int64_t &removed_cnt); + void check_all_redo_flushed(); int calc_checksum_before_scn(const share::SCN scn, - uint64_t &checksum, - share::SCN &checksum_scn); - void update_checksum(const uint64_t checksum, - const share::SCN checksum_scn); + ObIArray &checksum, + ObIArray &checksum_scn); + int update_checksum(const ObIArray &checksum, + const ObIArray &checksum_scn); int clean_unlog_callbacks(int64_t &removed_cnt); // when not inc, return -1 int64_t inc_pending_log_size(const int64_t size); void try_merge_multi_callback_lists(const int64_t new_size, const int64_t size, const bool is_logging_blocked); - void inc_flushed_log_size(const int64_t size) { UNUSED(ATOMIC_FAA(&flushed_log_size_, size)); } - void clear_pending_log_size() { ATOMIC_STORE(&pending_log_size_, 0); } - int64_t get_pending_log_size() { return ATOMIC_LOAD(&pending_log_size_); } - int64_t get_flushed_log_size() { return ATOMIC_LOAD(&flushed_log_size_); } - bool is_all_redo_submitted() - { - bool all_redo_submitted = true; - if (OB_NOT_NULL(callback_lists_)) { - for (int64_t i = 0; i < MAX_CALLBACK_LIST_COUNT; ++i) { - if (!callback_lists_[i].empty()) { - all_redo_submitted = false; - break; - } - } + void inc_flushed_log_size(const int64_t size) { + if (!serial_final_scn_.is_valid()) { + UNUSED(ATOMIC_FAA(&flushed_log_size_, size)); } - - if (all_redo_submitted) { - all_redo_submitted = !(((ObITransCallback *)callback_list_.get_tail())->need_submit_log()); - } - - return all_redo_submitted; } + void clear_pending_log_size() { ATOMIC_STORE(&pending_log_size_, 0); } + int64_t get_pending_log_size() const { return ATOMIC_LOAD(&pending_log_size_); } + bool pending_log_size_too_large(const transaction::ObTxSEQ &write_seq_no, const int64_t limit); + int64_t get_flushed_log_size() const { return ATOMIC_LOAD(&flushed_log_size_); } + int get_log_guard(const transaction::ObTxSEQ &write_seq, + ObCallbackListLogGuard &log_guard, + int &cb_list_idx); + int get_next_flush_log_guard(ObCallbackListLogGuard &lock_guard, int &list_idx); + void set_parallel_logging(const share::SCN serial_final_scn); + void set_skip_checksum_calc(); + bool skip_checksum_calc() const { return ATOMIC_LOAD(&skip_checksum_); } void merge_multi_callback_lists(); void reset_pdml_stat(); + bool find(ObITxCallbackFinder &func); uint64_t get_main_list_length() const { return callback_list_.get_length(); } int64_t get_callback_main_list_append_count() const @@ -299,18 +310,60 @@ public: { ATOMIC_AAF(&callback_remove_for_fast_commit_count_, cnt); } void add_rollback_to_callback_remove_cnt(int64_t cnt = 1) { ATOMIC_AAF(&callback_remove_for_rollback_to_count_, cnt); } - int64_t get_checksum() const { return callback_list_.get_checksum(); } - int64_t get_tmp_checksum() const { return callback_list_.get_tmp_checksum(); } - share::SCN get_checksum_scn() const { return callback_list_.get_checksum_scn(); } + int get_callback_list_count() const + { return callback_lists_ ? (MAX_CALLBACK_LIST_COUNT + (need_merge_ ? 1 : 0)) : 1; } + int get_logging_list_count() const; + ObTxCallbackList *get_callback_list_(const int16_t index, const bool nullable); + int is_callback_list_need_merge() const + { return need_merge_; } + bool is_serial_final() const { return is_serial_final_(); } + bool is_callback_list_append_only(const int idx) const + { + return (idx == 0) || !for_replay_ || is_serial_final_(); + } + void print_statistics(char *buf, const int64_t buf_len, int64_t &pos) const; transaction::ObPartTransCtx *get_trans_ctx() const; + TO_STRING_KV(KP(this), + K_(serial_final_scn), + K_(serial_sync_scn), + KP_(callback_lists), + K_(need_merge), + K_(pending_log_size), + K_(flushed_log_size), + K_(for_replay), + K_(parallel_stat)); private: void force_merge_multi_callback_lists(); -private: - ObITransCallback *get_guard_() { return callback_list_.get_guard(); } + void update_serial_sync_scn_(const share::SCN scn); + bool is_serial_final_() const + { + return serial_final_scn_ == serial_sync_scn_; + } + bool is_parallel_logging_() const + { + return !serial_final_scn_.is_max(); + } + int fill_from_one_list(ObTxFillRedoCtx &ctx, const int list_idx, ObITxFillRedoFunctor &func); + int fill_from_all_list(ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &func); + bool check_list_has_min_epoch_(const int my_idx, const int64_t my_epoch, int64_t &min_epoch, int &min_idx); + void calc_list_fill_log_epoch_(const int list_idx, int64_t &epoch_from, int64_t &epoch_to); + void calc_next_to_fill_log_info_(const ObIArray &arr, + int &index, + int64_t &epoch_from, + int64_t &epoch_to); + int prep_and_fill_from_list_(ObTxFillRedoCtx &ctx, + ObITxFillRedoFunctor &func, + int16 &callback_scope_idx, + const int index, + int64_t epoch_from, + int64_t epoch_to); private: ObIMvccCtx &host_; - ObTxCallbackList callback_list_; - ObTxCallbackList *callback_lists_; + // for incomplete replay, checksum is not need + // for tx is aborted, checksum is not need + bool skip_checksum_; + ObTxCallbackList callback_list_; // default + ObTxCallbackList *callback_lists_; // extends for parallel write common::SpinRWLock rwlock_; union { struct { @@ -319,7 +372,27 @@ private: }; int64_t parallel_stat_; }; + // multi writes at the same time is in a epoch + // used to serialize callbacks between multiple callback-list when fill redo + int64_t write_epoch_; + // remember the tid of first thread in current epoch + // the first thread is always assigned to first callback-list + int64_t write_epoch_start_tid_; + RLOCAL_STATIC(bool, parallel_replay_); + // since 4.3, support multi-callback-list logging, extended callback-list(s) + // won't do merge, use this to indicate the OLD version. + // it was set in write path. + // NOTE: this value is default set to false is required + // On Follower, in the OLD version, data is replay into single CallbackList + // On Leader, if no write after takeover, merge is also not required + bool need_merge_; bool for_replay_; + // the last serial logg's scn + share::SCN serial_final_scn_; + // currently synced serial log's scn + // when serial_sync_scn_ == serial_final_scn_ + // it means the serial logging or serial replay has been finished + share::SCN serial_sync_scn_; // statistics for callback remove int64_t callback_main_list_append_count_; int64_t callback_slave_list_append_count_; @@ -353,7 +426,7 @@ public: column_cnt_(0) {} ObMvccRowCallback(ObMvccRowCallback &cb, ObMemtable *memtable) : - ObITransCallback(cb.need_fill_redo_, cb.need_submit_log_), + ObITransCallback(cb.need_submit_log_), ctx_(cb.ctx_), value_(cb.value_), tnode_(cb.tnode_), @@ -418,11 +491,9 @@ public: int get_cluster_version(uint64_t &cluster_version) const override; transaction::ObTransCtx *get_trans_ctx() const; int64_t to_string(char *buf, const int64_t buf_len) const; - bool log_synced() const override { return share::SCN::max_scn() != scn_; } virtual int before_append(const bool is_replay) override; virtual void after_append(const bool is_replay) override; - virtual int log_submitted() override; - virtual int undo_log_submitted() override; + virtual int log_submitted(const share::SCN scn, ObIMemtable *&last_mt) override; int64_t get_data_size() { return data_size_; @@ -430,8 +501,7 @@ public: virtual int clean(); virtual int del(); virtual int checkpoint_callback(); - virtual int log_sync(const share::SCN scn) override; - virtual int log_sync_fail() override; + virtual int log_sync_fail(const share::SCN max_applied_scn) override; virtual int print_callback() override; virtual blocksstable::ObDmlFlag get_dml_flag() const override; virtual void set_not_calc_checksum(const bool not_calc_checksum) override @@ -445,7 +515,7 @@ private: virtual int trans_abort() override; virtual int rollback_callback() override; virtual int calc_checksum(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) override; + TxChecksum *checksumer) override; virtual int elr_trans_preparing() override; private: int link_and_get_next_node(ObMvccTransNode *&next); @@ -454,9 +524,7 @@ private: ObMemtableKey &memtable_key, const common::ObTabletID &tablet_id); int clean_unlog_cb(); void inc_unsubmitted_cnt_(); - void inc_unsynced_cnt_(); int dec_unsubmitted_cnt_(); - int dec_unsynced_cnt_(); int wakeup_row_waiter_if_need_(); private: ObIMvccCtx &ctx_; diff --git a/src/storage/memtable/mvcc/ob_query_engine.cpp b/src/storage/memtable/mvcc/ob_query_engine.cpp index 328ea641c8..efc2e48a66 100644 --- a/src/storage/memtable/mvcc/ob_query_engine.cpp +++ b/src/storage/memtable/mvcc/ob_query_engine.cpp @@ -235,7 +235,7 @@ int ObQueryEngine::get(const ObMemtableKey *parameter_key, ObMvccRow *&row, ObMe } else if (OB_ISNULL(row)) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "get NULL value from keyhash", KR(ret), K(*parameter_key)); - } else { + } else if (returned_key) { ret = returned_key->encode(copy_inner_key_wrapper->get_rowkey()); } } diff --git a/src/storage/memtable/mvcc/ob_tx_callback_functor.h b/src/storage/memtable/mvcc/ob_tx_callback_functor.h index e0112745fd..18a194bef3 100644 --- a/src/storage/memtable/mvcc/ob_tx_callback_functor.h +++ b/src/storage/memtable/mvcc/ob_tx_callback_functor.h @@ -15,12 +15,16 @@ #include "lib/function/ob_function.h" #include "storage/memtable/mvcc/ob_mvcc.h" - namespace oceanbase { +namespace transaction +{ +class ObCLogEncryptInfo; +} namespace memtable { - +class ObMutatorWriter; +class ObTxFillRedoCtx; class ObITxCallbackFunctor { public: @@ -72,14 +76,12 @@ protected: class ObRemoveCallbacksForFastCommitFunctor : public ObITxCallbackFunctor { public: - ObRemoveCallbacksForFastCommitFunctor(const ObITransCallback *generate_cursor, - const int64_t need_remove_count) - : generate_cursor_(generate_cursor), - need_remove_count_(need_remove_count), + ObRemoveCallbacksForFastCommitFunctor(const int64_t need_remove_count, const share::SCN &sync_scn) + : need_remove_count_(need_remove_count), + sync_scn_(sync_scn), last_scn_for_remove_(share::SCN::min_scn()), checksum_scn_(share::SCN::min_scn()), - checksumer_(NULL), - meet_generate_cursor_(false) {} + checksumer_(NULL) {} virtual bool is_iter_end(ObITransCallback *callback) const override { @@ -88,22 +90,19 @@ public: if (NULL == callback) { // case1: the callback is nullptr is_iter_end = true; - } else if (callback->need_fill_redo() || callback->need_submit_log()) { + } else if (callback->need_submit_log()) { // case2: the callback has not been sync successfully is_iter_end = true; - } else if (share::SCN::max_scn() == callback->get_scn()) { + } else if (sync_scn_ < callback->get_scn()) { // case3: the callback has not been sync successfully is_iter_end = true; - } else if (share::SCN::min_scn() != last_scn_for_remove_ - && callback->get_scn() != last_scn_for_remove_) { + } else if (callback->get_scn().is_min()) { + usleep(5000); + ob_abort(); + } else if (0 >= need_remove_count_ && callback->get_scn() != last_scn_for_remove_) { // case4: the callback has exceeded the last log whose log ts need to be // removed is_iter_end = true; - } else if (share::SCN::min_scn() == last_scn_for_remove_ - && 0 >= need_remove_count_) { - // case6: the callback has not reached the last log whose log ts need to - // be removed while having no logs need to be removed - is_iter_end = true; } else { is_iter_end = false; } @@ -112,7 +111,7 @@ public: } void set_checksumer(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) + TxChecksum *checksumer) { checksum_scn_ = checksum_scn; checksumer_ = checksumer; @@ -126,89 +125,41 @@ public: int operator()(ObITransCallback *callback) { int ret = OB_SUCCESS; - - if (NULL == checksumer_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "checksumer is lost", K(ret), K(*callback)); - } else if (callback->get_scn() >= checksum_scn_ - && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { + if (checksumer_ && callback->get_scn() >= checksum_scn_ + && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { TRANS_LOG(WARN, "calc checksum callback failed", K(ret), K(*callback)); } else if (OB_FAIL(callback->checkpoint_callback())) { TRANS_LOG(ERROR, "row remove callback failed", K(ret), K(*callback)); } else { need_remove_callback_ = true; - need_remove_count_--; - - // If we are removing callback pointed by generate_cursor, we need reset - // the generate_cursor. Otherwise the dangling pointer may coredump. - if (generate_cursor_ == callback) { - meet_generate_cursor_ = true; - } - - // if we satisfy the removing count of fast commit, we still need remember - // the last log ts we have already removed and then continue to remove the - // callbacks until all callbacks with the same log ts has been removed in - // order to satisfy the checksum calculation - ObITransCallback *next = callback->get_next(); - if (share::SCN::min_scn() == last_scn_for_remove_) { - if (0 == need_remove_count_) { - last_scn_for_remove_ = callback->get_scn(); - } else if (NULL == next - || next->need_submit_log() - || next->need_fill_redo() - || share::SCN::max_scn() == next->get_scn()) { - last_scn_for_remove_ = callback->get_scn(); - } - } + --need_remove_count_; + last_scn_for_remove_ = callback->get_scn(); } return ret; } - // return whether we are removing callbacks that pointed by generate cursor - bool meet_generate_cursor() const - { - return meet_generate_cursor_; - } - - VIRTUAL_TO_STRING_KV(KP_(generate_cursor), - K_(need_remove_count), + VIRTUAL_TO_STRING_KV(K_(need_remove_count), + KP_(checksumer), + K_(sync_scn), K_(checksum_scn), - K_(last_scn_for_remove), - K_(meet_generate_cursor)); + K_(last_scn_for_remove)); private: - const ObITransCallback *generate_cursor_; int64_t need_remove_count_; + share::SCN sync_scn_; share::SCN last_scn_for_remove_; share::SCN checksum_scn_; - ObBatchChecksum *checksumer_; - - bool meet_generate_cursor_; -}; - -class ObNeverStopForCallbackTraverseFunctor -{ -public: - bool operator()(ObITransCallback *callback) - { - UNUSED(callback); - return false; - } + TxChecksum *checksumer_; }; // TODO(handora.qc): Adapt to ObRemoveCallbacksWCondFunctor class ObRemoveSyncCallbacksWCondFunctor : public ObITxCallbackFunctor { public: - ObRemoveSyncCallbacksWCondFunctor( - const ObFunction &remove_func, - const ObFunction &stop_func = ObNeverStopForCallbackTraverseFunctor(), - const bool need_remove_data = true, - const bool is_reverse = false) - : cond_for_remove_(remove_func), - cond_for_stop_(stop_func), - need_checksum_(false), + ObRemoveSyncCallbacksWCondFunctor(const bool need_remove_data = true, + const bool is_reverse = false) + : need_checksum_(false), need_remove_data_(need_remove_data), checksum_scn_(share::SCN::min_scn()), checksumer_(NULL), @@ -216,7 +167,12 @@ public: { is_reverse_ = is_reverse; } - + virtual bool cond_for_remove(ObITransCallback *callback) = 0; + virtual bool cond_for_stop(ObITransCallback *callback) const + { + UNUSED(callback); + return false; + }; bool check_valid_() { bool is_valid = true; @@ -230,7 +186,8 @@ public: } void set_checksumer(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) + TxChecksum *checksumer + ) { need_checksum_ = true; checksum_scn_ = checksum_scn; @@ -242,8 +199,7 @@ public: if (need_checksum_) { return checksum_last_scn_; } else { - TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "we donot go here if we donot checksum", KPC(this)); - return share::SCN::min_scn(); + return share::SCN::invalid_scn(); } } @@ -254,13 +210,13 @@ public: if (NULL == callback) { // case1: the callback is nullptr is_iter_end = true; - } else if (callback->need_fill_redo() || callback->need_submit_log()) { + } else if (callback->need_submit_log()) { // case2: the callback has not been sync successfully is_iter_end = true; } else if (share::SCN::max_scn() == callback->get_scn()) { // case3: the callback has not been sync successfully is_iter_end = true; - } else if (cond_for_stop_(callback)) { + } else if (cond_for_stop(callback)) { // case4: user want to stop here is_iter_end = true; } else { @@ -273,19 +229,20 @@ public: int operator()(ObITransCallback *callback) { int ret = OB_SUCCESS; - if (!check_valid_()) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "we cannot calc checksum when reverse remove", K(ret), KPC(this)); - } else if (callback->need_fill_redo() || callback->need_submit_log()) { + } else if (callback->need_submit_log()) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "remove synced will never go here", K(ret), KPC(callback)); - } else if (need_checksum_ - && callback->get_scn() >= checksum_scn_ - && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { + } else if (need_checksum_ && callback->get_scn() >= checksum_scn_ ) { + if (OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { TRANS_LOG(WARN, "row remove callback failed", K(ret), K(*callback)); - } else if (FALSE_IT(checksum_last_scn_ = callback->get_scn())) { - } else if (cond_for_remove_(callback)) { + } else if (FALSE_IT(checksum_last_scn_ = callback->get_scn())) { + } + } + if (OB_FAIL(ret)) { + } else if (cond_for_remove(callback)) { if (need_remove_data_ && OB_FAIL(callback->rollback_callback())) { TRANS_LOG(WARN, "rollback callback failed", K(ret), K(*callback)); } else if (!need_remove_data_ && OB_FAIL(callback->checkpoint_callback())) { @@ -300,32 +257,31 @@ public: VIRTUAL_TO_STRING_KV(K_(need_checksum), K_(need_remove_data), + K_(remove_cnt), K_(checksum_scn), K_(checksum_last_scn)); private: - ObFunction cond_for_remove_; - ObFunction cond_for_stop_; bool need_checksum_; bool need_remove_data_; share::SCN checksum_scn_; - ObBatchChecksum *checksumer_; + TxChecksum *checksumer_; share::SCN checksum_last_scn_; }; class ObRemoveCallbacksWCondFunctor : public ObITxCallbackFunctor { public: - ObRemoveCallbacksWCondFunctor(ObFunction func, - const bool need_remove_data = true) - : cond_for_remove_(func), - need_remove_data_(need_remove_data), + ObRemoveCallbacksWCondFunctor(const share::SCN right_bound, const bool need_remove_data = true) + : need_remove_data_(need_remove_data), + right_bound_(right_bound), checksum_scn_(share::SCN::min_scn()), checksumer_(NULL), checksum_last_scn_(share::SCN::min_scn()) {} - + virtual bool cond_for_remove(ObITransCallback* callback) = 0; void set_checksumer(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) + TxChecksum *checksumer + ) { checksum_scn_ = checksum_scn; checksumer_ = checksumer; @@ -336,11 +292,12 @@ public: if (NULL != checksumer_) { return checksum_last_scn_; } else { - TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "we donot go here if we donot checksum", KPC(this)); - return share::SCN::min_scn(); + return share::SCN::invalid_scn(); } } - + bool is_iter_end(ObITransCallback *callback) const { + return callback->get_scn() > right_bound_; + } int operator()(ObITransCallback *callback) { int ret = OB_SUCCESS; @@ -348,17 +305,9 @@ public: if (NULL == callback) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "unexpected callback", KP(callback)); - } else if (!callback->need_fill_redo() && callback->need_submit_log()) { - // Case 1: callback synced before proposed to paxos - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "It will never on success before submit log", KPC(callback)); - } else if (callback->need_fill_redo() && !callback->need_submit_log()) { - // Case 2: callback has been submitted but not be synced - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "callbacks cannot be removed before synced", KPC(callback)); - } else if (callback->need_fill_redo() && callback->need_submit_log()) { - // Case 3: callback has not been proposed to paxos - if (cond_for_remove_(callback)) { + } else if (callback->need_submit_log()) { + // Case 1: callback has not been proposed to paxos + if (cond_for_remove(callback)) { if (need_remove_data_ && OB_FAIL(callback->rollback_callback())) { TRANS_LOG(WARN, "rollback callback failed", K(ret), K(*callback)); } else if (!need_remove_data_ && OB_FAIL(callback->checkpoint_callback())) { @@ -367,14 +316,12 @@ public: need_remove_callback_ = true; } } - } else if (!callback->need_fill_redo() && !callback->need_submit_log()) { - // Case 4: callback has synced successfully - if (cond_for_remove_(callback)) { - if (NULL == checksumer_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "checksumer is lost", K(ret), K(*callback)); - } else if (callback->get_scn() >= checksum_scn_ - && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { + } else if (!callback->need_submit_log()) { + // Case 2: callback has submitted to log-service may not persistented + // we check removable in cond_for_remove_ ensure it is synced + if (cond_for_remove(callback)) { + if (checksumer_ && callback->get_scn() >= checksum_scn_ + && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { TRANS_LOG(WARN, "calc checksum callback failed", K(ret), K(*callback)); } else if (need_remove_data_ && OB_FAIL(callback->rollback_callback())) { TRANS_LOG(WARN, "rollback callback failed", K(ret), K(*callback)); @@ -382,17 +329,18 @@ public: TRANS_LOG(WARN, "checkpoint callback failed", K(ret), K(*callback)); } else { need_remove_callback_ = true; - checksum_last_scn_ = callback->get_scn(); + if (checksumer_) { + checksum_last_scn_ = callback->get_scn(); + } } } else { - if (NULL == checksumer_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "checksumer is lost", K(ret), K(*callback)); - } else if (callback->get_scn() >= checksum_scn_ - && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { - TRANS_LOG(WARN, "row remove callback failed", K(ret), K(*callback)); - } else { - checksum_last_scn_ = callback->get_scn(); + if (checksumer_) { + if (callback->get_scn() >= checksum_scn_ + && OB_FAIL(callback->calc_checksum(checksum_scn_, checksumer_))) { + TRANS_LOG(WARN, "row remove callback failed", K(ret), K(*callback)); + } else { + checksum_last_scn_ = callback->get_scn(); + } } } } @@ -401,14 +349,16 @@ public: } VIRTUAL_TO_STRING_KV(K_(need_remove_data), + K_(right_bound), + KP_(checksumer), K_(checksum_scn), K_(checksum_last_scn)); private: - ObFunction cond_for_remove_; bool need_remove_data_; + share::SCN right_bound_; share::SCN checksum_scn_; - ObBatchChecksum *checksumer_; + TxChecksum *checksumer_; share::SCN checksum_last_scn_; }; @@ -444,9 +394,11 @@ public: TRANS_LOG(ERROR, "unexpected callback", KP(callback)); /* } else if (is_commit_ && */ /* (callback->need_submit_log() */ - /* || callback->need_fill_redo())) { */ /* ret = OB_ERR_UNEXPECTED; */ /* TRANS_LOG(ERROR, "unexpected callback", KP(callback)); */ + } else if (is_commit_ && callback->get_scn().is_max()) { + TRANS_LOG(ERROR, "callback has not submitted log yet when commit callback", KP(callback)); + ob_abort(); } else if (is_commit_ && OB_FAIL(callback->trans_commit())) { TRANS_LOG(ERROR, "trans commit failed", KPC(callback)); @@ -473,7 +425,6 @@ private: if (!is_commit_ && !need_print_ && callback->need_submit_log() && - callback->need_fill_redo() && callback->is_logging_blocked()) { need_print_ = true; } @@ -500,13 +451,7 @@ public: if (NULL == callback) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "unexpected callback", KP(callback)); - } else if (callback->need_fill_redo() && !callback->need_submit_log()) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "all callbacks must be invoked before leader switch", KPC(callback)); - } else if (!callback->need_fill_redo() && callback->need_submit_log()) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "It will never on success before submit log", KPC(callback)); - } else if (callback->need_fill_redo() && callback->need_submit_log()) { + } else if (callback->need_submit_log()) { callback->rollback_callback(); need_remove_callback_ = true; } @@ -529,17 +474,8 @@ public: if (NULL == callback) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "unexpected callback", KP(callback)); - } else if (callback->need_fill_redo() && callback->need_submit_log()) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "sync log fail will only touch submitted log", KPC(callback)); - } else if (!callback->need_fill_redo() && !callback->need_submit_log()) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "sync log fail will only touch unsynced log", KPC(callback)); - } else if (!callback->need_fill_redo() && callback->need_submit_log()) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "It will never on success before submit log", KPC(callback)); - } else if (callback->need_fill_redo() && !callback->need_submit_log()) { - if (OB_FAIL(callback->log_sync_fail_cb())) { + } else if (!callback->need_submit_log()) { // log has been submitted out + if (OB_FAIL(callback->log_sync_fail_cb(max_committed_scn_))) { // log_sync_fail_cb will never report error TRANS_LOG(ERROR, "log sync fail cb report error", K(ret)); } else { @@ -549,8 +485,8 @@ public: return ret; } - - VIRTUAL_TO_STRING_KV("ObSyncLogFailFunctor", "ObSyncLogFailFunctor"); + share::SCN max_committed_scn_; + TO_STRING_KV(K_(max_committed_scn)); }; class ObSearchCallbackWCondFunctor : public ObITxCallbackFunctor @@ -570,7 +506,7 @@ public: return NULL != search_res_; } - virtual int operator()(ObITransCallback *callback) override + virtual int operator()(ObITransCallback *callback) override { int ret = OB_SUCCESS; @@ -603,7 +539,7 @@ public: return ObTimeUtility::current_time() - start_ts_ >= 300 * 1000L; } - virtual int operator()(ObITransCallback *callback) override + virtual int operator()(ObITransCallback *callback) override { int ret = OB_SUCCESS; @@ -649,7 +585,7 @@ public: void set_checksumer(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) + TxChecksum *checksumer) { checksum_scn_ = checksum_scn; checksumer_ = checksumer; @@ -663,7 +599,6 @@ public: int operator()(ObITransCallback *callback) { int ret = OB_SUCCESS; - if (NULL == checksumer_) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "checksumer is lost", K(ret), K(*callback)); @@ -686,10 +621,38 @@ public: private: share::SCN target_scn_; share::SCN checksum_scn_; - ObBatchChecksum *checksumer_; + TxChecksum *checksumer_; share::SCN checksum_last_scn_; }; +class ObITxFillRedoFunctor : public ObITxCallbackFunctor +{ +public: + int operator()(ObITransCallback *iter) = 0; + int64_t get_data_size() const { return data_size_; } + transaction::ObTxSEQ get_max_seq_no() const { return max_seq_no_; } + void reset() { + data_size_ = 0; + max_seq_no_.reset(); + } +protected: + int64_t data_size_; + transaction::ObTxSEQ max_seq_no_; +}; + +class ObITxCallbackFinder : public ObITxCallbackFunctor { +public: + ObITxCallbackFinder() : found_(false) {} + virtual bool match(ObITransCallback *callback) = 0; + int operator()(ObITransCallback *callback) { + found_ = match(callback); + return OB_SUCCESS; + } + bool is_iter_end(ObITransCallback *callback) const { return found_; } + bool is_found() const { return found_; } +private: + bool found_; +}; } // memtable } // oceanbase diff --git a/src/storage/memtable/mvcc/ob_tx_callback_list.cpp b/src/storage/memtable/mvcc/ob_tx_callback_list.cpp index e57c919ac2..6a08500259 100644 --- a/src/storage/memtable/mvcc/ob_tx_callback_list.cpp +++ b/src/storage/memtable/mvcc/ob_tx_callback_list.cpp @@ -9,13 +9,13 @@ * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. * See the Mulan PubL v2 for more details. */ - #include "storage/memtable/mvcc/ob_tx_callback_list.h" #include "storage/memtable/mvcc/ob_mvcc_ctx.h" #include "share/config/ob_server_config.h" #include "storage/memtable/ob_memtable_key.h" #include "storage/tx/ob_trans_define.h" #include "storage/tx/ob_trans_part_ctx.h" +#include "storage/tx/ob_tx_stat.h" namespace oceanbase { @@ -23,50 +23,102 @@ using namespace share; namespace memtable { -ObTxCallbackList::ObTxCallbackList(ObTransCallbackMgr &callback_mgr) - : head_(), +ObTxCallbackList::ObTxCallbackList(ObTransCallbackMgr &callback_mgr, const int16_t id) + : id_(id), + head_(), + log_cursor_(&head_), + parallel_start_pos_(NULL), length_(0), + appended_(0), + logged_(0), + synced_(0), + removed_(0), + unlog_removed_(0), + branch_removed_(0), + data_size_(0), + logged_data_size_(0), + sync_scn_(SCN::min_scn()), batch_checksum_(), checksum_scn_(SCN::min_scn()), checksum_(0), tmp_checksum_(0), callback_mgr_(callback_mgr), - latch_() -{ reset(); } + append_latch_(), + log_latch_(), + iter_synced_latch_() +{ + reset(); +} ObTxCallbackList::~ObTxCallbackList() {} void ObTxCallbackList::reset() { + if (length_ + removed_ != appended_) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "BUG:list state insanity", KPC(this)); + ob_abort(); + } + if (length_ + removed_ != logged_ + unlog_removed_) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "BUG:list state insanity", KPC(this)); + ob_abort(); + } head_.set_prev(&head_); head_.set_next(&head_); + log_cursor_ = &head_; checksum_ = 0; tmp_checksum_ = 0; checksum_scn_ = SCN::min_scn(); batch_checksum_.reset(); length_ = 0; + appended_ = 0; + logged_ = 0; + synced_ = 0; + removed_ = 0; + unlog_removed_ = 0; + branch_removed_ = 0; + data_size_ = 0; + logged_data_size_ = 0; + sync_scn_ = SCN::min_scn(); } // the semantic of the append_callback is atomic which means the cb is removed // and no side effect is taken effects if some unexpected failure has happened. int ObTxCallbackList::append_callback(ObITransCallback *callback, - const bool for_replay) + const bool for_replay, + const bool parallel_replay, + const bool serial_final) { int ret = OB_SUCCESS; // It is important that we should put the before_append_cb and after_append_cb // into the latch guard otherwise the callback may already paxosed and released // before callback it. - ObByteLockGuard guard(latch_); - + LockGuard gaurd(*this, LOCK_MODE::LOCK_APPEND); if (OB_ISNULL(callback)) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "before_append_cb failed", K(ret), KPC(callback)); } else if (OB_FAIL(callback->before_append_cb(for_replay))) { TRANS_LOG(WARN, "before_append_cb failed", K(ret), KPC(callback)); } else { - (void)get_tail()->append(callback); - length_++; - + const bool repos_lc = !for_replay && (log_cursor_ == &head_); + if (!for_replay || parallel_replay || serial_final || !parallel_start_pos_) { + (void)get_tail()->append(callback); + } else { + parallel_start_pos_->get_prev()->append(callback); + } + // start parallel replay, remember the position + if (for_replay && parallel_replay && !serial_final && !parallel_start_pos_) { + ATOMIC_STORE(¶llel_start_pos_, get_tail()); + } + ++appended_; + ATOMIC_INC(&length_); + data_size_ += callback->get_data_size(); + if (repos_lc) { + log_cursor_ = get_tail(); + } + if (for_replay) { + ++logged_; + ++synced_; + } // Once callback is appended into callback lists, we can not handle the // error after it. So it should never report the error later. What's more, // after_append also should never return the error. @@ -83,8 +135,8 @@ int64_t ObTxCallbackList::concat_callbacks(ObTxCallbackList &that) if (that.empty()) { // do nothing } else { - ObByteLockGuard this_guard(latch_); - ObByteLockGuard that_guard(that.latch_); + LockGuard this_guard(*this, LOCK_MODE::LOCK_ALL); + LockGuard that_guard(that, LOCK_MODE::LOCK_ALL); ObITransCallback *that_head = that.head_.get_next(); ObITransCallback *that_tail = that.get_tail(); that_head->set_prev(get_tail()); @@ -93,32 +145,43 @@ int64_t ObTxCallbackList::concat_callbacks(ObTxCallbackList &that) head_.set_prev(that_tail); cnt = that.get_length(); length_ += cnt; - that.reset(); + appended_ += cnt; + if (log_cursor_ == &head_) { + log_cursor_ = that_head; + } + { // fake callback removement to pass sanity check when reset + that.length_ = 0; + that.removed_ = cnt; + that.unlog_removed_ = cnt; + that.reset(); + } } return cnt; } -int ObTxCallbackList::callback_(ObITxCallbackFunctor &functor) +int ObTxCallbackList::callback_(ObITxCallbackFunctor &functor, const LockState lock_state) { - return callback_(functor, get_guard(), get_guard()); + return callback_(functor, get_guard(), get_guard(), lock_state); } int ObTxCallbackList::callback_(ObITxCallbackFunctor &functor, - const ObCallbackScope &callbacks) + const ObCallbackScope &callbacks, + const LockState lock_state) { ObITransCallback *start = (ObITransCallback *)*(callbacks.start_); ObITransCallback *end = (ObITransCallback *)*(callbacks.end_); if (functor.is_reverse()) { - return callback_(functor, start->get_next(), end->get_prev()); + return callback_(functor, start->get_next(), end->get_prev(), lock_state); } else { - return callback_(functor, start->get_prev(), end->get_next()); + return callback_(functor, start->get_prev(), end->get_next(), lock_state); } } int ObTxCallbackList::callback_(ObITxCallbackFunctor &functor, - ObITransCallback *start, - ObITransCallback *end) + ObITransCallback *start /*exclusive*/, + ObITransCallback *end /*exclusive*/, + const LockState lock_state) { int ret = OB_SUCCESS; int64_t traverse_count = 0; @@ -126,45 +189,85 @@ int ObTxCallbackList::callback_(ObITxCallbackFunctor &functor, ObITransCallback *next = nullptr; bool iter_end = false; const bool is_reverse = functor.is_reverse(); - for (ObITransCallback *iter = (is_reverse ? start->get_prev() : start->get_next()); OB_SUCC(ret) && !iter_end && iter != NULL && iter != end; iter = next) { functor.refresh(); - + if (iter->get_scn().is_min()) { + usleep(5000); + ob_abort(); + } if (functor.is_iter_end(iter)) { iter_end = true; } else { next = (is_reverse ? iter->get_prev() : iter->get_next()); - if (OB_FAIL(functor(iter))) { - TRANS_LOG(WARN, "functor call failed", KPC(iter)); + // don't print log, print it in functor } else if (functor.need_remove_callback()) { - if (OB_FAIL(iter->del())) { - TRANS_LOG(ERROR, "remove callback failed", KPC(iter)); + if (removed_ && (removed_ % 1000 == 0)) { + uint64_t checksum_now = batch_checksum_.calc(); + TRANS_LOG(INFO, "[CallbackList] remove-callback", K(checksum_now), KPC(this)); + } + const share::SCN iter_scn = iter->get_scn(); + // the del operation must serialize with append operation + // if it is operating on the list tail + bool deleted = false; + if (next == end) { + if (!lock_state.APPEND_LOCKED_) { + LockGuard guard(*this, LOCK_MODE::LOCK_APPEND); + ret = iter->del(); + deleted = true; + } else { + ret = iter->del(); + deleted = true; + } + } + if ((deleted && OB_FAIL(ret)) || (!deleted && OB_FAIL(iter->del()))) { + TRANS_LOG(ERROR, "remove callback failed", K(ret), KPC(iter), K(deleted)); } else { + if (parallel_start_pos_ + && !is_skip_checksum_() + && !callback_mgr_.is_serial_final() + && iter_scn >= parallel_start_pos_->get_scn() + && iter_scn <= sync_scn_) { + TRANS_LOG(ERROR, "should not remove this callback", KPC(iter)); + usleep(5000); + ob_abort(); + } + if (log_cursor_ == iter) { + log_cursor_ = next; + } + if (parallel_start_pos_ == iter) { + parallel_start_pos_ = (next == &head_) ? NULL : next; + } + ++removed_; + if (iter->need_submit_log()) { + ++unlog_removed_; + } + + ++remove_count; + if (iter->is_need_free()) { callback_mgr_.get_ctx().callback_free(iter); } - remove_count++; } } if ((++traverse_count & 0xFFFFF) == 0) { TRANS_LOG(WARN, "memtable fifo callback too long", - K(traverse_count), K(functor)); + K(traverse_count), K(remove_count), K(functor)); } } } - functor.set_statistics(traverse_count, remove_count); - length_ -= remove_count; - + ATOMIC_AAF(&length_, -remove_count); return ret; } int64_t ObTxCallbackList::calc_need_remove_count_for_fast_commit_() { + // TODO: when support multiple callback-list, the remove count for single + // list should be re-developed const int64_t fast_commit_callback_count = GCONF._fast_commit_callback_count; const int64_t recommand_reserve_count = (fast_commit_callback_count + 1) / 2; const int64_t need_remove_count = length_ - recommand_reserve_count; @@ -172,43 +275,54 @@ int64_t ObTxCallbackList::calc_need_remove_count_for_fast_commit_() return need_remove_count; } -int ObTxCallbackList::remove_callbacks_for_fast_commit(const ObITransCallback *generate_cursor, - bool &meet_generate_cursor) +int ObTxCallbackList::remove_callbacks_for_fast_commit(const share::SCN stop_scn) { int ret = OB_SUCCESS; - meet_generate_cursor = false; - ObByteLockGuard guard(latch_); - - ObRemoveCallbacksForFastCommitFunctor functor(generate_cursor, - calc_need_remove_count_for_fast_commit_()); - functor.set_checksumer(checksum_scn_, &batch_checksum_); - - if (OB_FAIL(callback_(functor))) { - TRANS_LOG(ERROR, "remove callbacks for fast commit wont report error", K(ret), K(functor)); - } else { - callback_mgr_.add_fast_commit_callback_remove_cnt(functor.get_remove_cnt()); - ensure_checksum_(functor.get_checksum_last_scn()); - meet_generate_cursor = functor.meet_generate_cursor(); + // if one thread doing the fast-commit, others give up + LockGuard guard(*this, LOCK_MODE::TRY_LOCK_ITERATE); + if (guard.is_locked()) { + int64_t remove_cnt = calc_need_remove_count_for_fast_commit_(); + // use rm_logged_latch_ to protected multiple thread try to remove logged callbacks + bool skip_checksum = is_skip_checksum_(); + const share::SCN right_bound = skip_checksum ? share::SCN::max_scn() + : (stop_scn.is_valid() ? stop_scn : sync_scn_); + ObRemoveCallbacksForFastCommitFunctor functor(remove_cnt, right_bound); + if (!skip_checksum) { + functor.set_checksumer(checksum_scn_, &batch_checksum_); + } + if (OB_FAIL(callback_(functor, get_guard(), log_cursor_, guard.state_))) { + TRANS_LOG(ERROR, "remove callbacks for fast commit wont report error", K(ret), K(functor)); + } else { + callback_mgr_.add_fast_commit_callback_remove_cnt(functor.get_remove_cnt()); + ensure_checksum_(functor.get_checksum_last_scn()); + } } - return ret; } int ObTxCallbackList::remove_callbacks_for_remove_memtable( const memtable::ObMemtableSet *memtable_set, - const share::SCN max_applied_scn) + const share::SCN stop_scn) { + // remove callbacks for remove_memtable imply the + // callbacks must have been synced (because of the memtable must has been checkpointed) + // so, it is safe to stop at sync_scn_ + // this operation also comes through TxCtx, and has hold TxCtx's whole lock + // hence, acquire iter_latch is not required actually. int ret = OB_SUCCESS; - ObByteLockGuard guard(latch_); - - ObRemoveSyncCallbacksWCondFunctor functor( - // condition for remove - [memtable_set](ObITransCallback *callback) -> bool { + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); + const bool skip_checksum = is_skip_checksum_(); + const share::SCN right_bound = skip_checksum ? share::SCN::max_scn() + : (stop_scn.is_max() ? sync_scn_ : stop_scn); + struct Functor final : public ObRemoveSyncCallbacksWCondFunctor { + Functor(const bool need_remove_data = true, const bool is_reverse = false) + : ObRemoveSyncCallbacksWCondFunctor(need_remove_data, is_reverse) {} + bool cond_for_remove(ObITransCallback *callback) { bool ok = false; int ret = OB_SUCCESS; int bool_ret = true; while (!ok) { - if (OB_HASH_EXIST == (ret = memtable_set->exist_refactored((uint64_t)callback->get_memtable()))) { + if (OB_HASH_EXIST == (ret = memtable_set_->exist_refactored((uint64_t)callback->get_memtable()))) { bool_ret = true; ok = true; } else if (OB_HASH_NOT_EXIST == ret) { @@ -221,54 +335,82 @@ int ObTxCallbackList::remove_callbacks_for_remove_memtable( } } return bool_ret; - }, // condition for stop - [max_applied_scn](ObITransCallback *callback) -> bool { - if (callback->get_scn() > max_applied_scn) { - return true; - } else { - return false; - } - }, - false /*need_remove_data*/); - functor.set_checksumer(checksum_scn_, &batch_checksum_); + } + bool cond_for_stop(ObITransCallback *callback) const { + return callback->get_scn() > right_bound_; + } + share::SCN right_bound_; + const memtable::ObMemtableSet *memtable_set_; + } functor(false); - if (OB_FAIL(callback_(functor))) { + functor.right_bound_ = right_bound; + functor.memtable_set_ = memtable_set; + + if (!skip_checksum) { + functor.set_checksumer(checksum_scn_, &batch_checksum_); + } + + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(ERROR, "remove callbacks for remove memtable wont report error", K(ret), K(functor)); } else { callback_mgr_.add_release_memtable_callback_remove_cnt(functor.get_remove_cnt()); ensure_checksum_(functor.get_checksum_last_scn()); if (functor.get_remove_cnt() > 0) { TRANS_LOG(INFO, "remove callbacks for remove memtable", KP(memtable_set), - K(functor), K(*this)); + K(functor.get_remove_cnt()), K(stop_scn), K(right_bound), K(functor), K(*this)); } } return ret; } -int ObTxCallbackList::remove_callbacks_for_rollback_to(const transaction::ObTxSEQ to_seq_no) +// when remove callback for rollback to in replay +// the caller has promise the replay of callback list is prefix completed +// especially for branch level savepoint rollback in parallel replay situation +int ObTxCallbackList::remove_callbacks_for_rollback_to(const transaction::ObTxSEQ to_seq, + const transaction::ObTxSEQ from_seq, + const share::SCN replay_scn) { int ret = OB_SUCCESS; - ObByteLockGuard guard(latch_); + // because of rollback_to has acquire TxCtx's whole lock, it can prevent all operations + // through TxCtx + // acquire append_latch_ to prevent append, this is because of + // table-lock will append callback directly to callback-list passthrough TxCtx's + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + const share::SCN right_bound = replay_scn.is_valid() ? replay_scn : share::SCN::max_scn(); + struct Functor final : public ObRemoveCallbacksWCondFunctor { + Functor(const share::SCN right_bound, const bool need_remove_data = true) + : ObRemoveCallbacksWCondFunctor(right_bound, need_remove_data) {} + bool cond_for_remove(ObITransCallback *callback) { + transaction::ObTxSEQ dseq = callback->get_seq_no(); + // sanity check + OB_ASSERT(to_seq_.support_branch() == dseq.support_branch()); + return (to_seq_.get_branch() == 0 // match all branches + || to_seq_.get_branch() == dseq.get_branch()) // match target branch + && dseq.get_seq() > to_seq_.get_seq() // exclusive + && dseq.get_seq() < from_seq_.get_seq(); // inclusive + } + transaction::ObTxSEQ to_seq_; + transaction::ObTxSEQ from_seq_; + } functor(right_bound, true); - ObRemoveCallbacksWCondFunctor functor( - [to_seq_no](ObITransCallback *callback) -> bool { - if (callback->get_seq_no() > to_seq_no) { - return true; - } else { - return false; - } - }, true/*need_remove_data*/); - functor.set_checksumer(checksum_scn_, &batch_checksum_); + functor.to_seq_ = to_seq; + functor.from_seq_ = from_seq; - if (OB_FAIL(callback_(functor))) { + if(!is_skip_checksum_()) { + functor.set_checksumer(checksum_scn_, &batch_checksum_); + } + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(ERROR, "remove callback by rollback wont report error", K(ret), K(functor)); } else { - callback_mgr_.add_rollback_to_callback_remove_cnt(functor.get_remove_cnt()); + int64_t removed = functor.get_remove_cnt(); + if (to_seq.support_branch() && to_seq.get_branch()) { + branch_removed_ += removed; + } + callback_mgr_.add_rollback_to_callback_remove_cnt(removed); ensure_checksum_(functor.get_checksum_last_scn()); - TRANS_LOG(DEBUG, "remove callbacks for rollback to", K(to_seq_no), K(functor), K(*this)); + TRANS_LOG(TRACE, "remove callbacks for rollback to", K(to_seq), K(from_seq), K(removed), K(functor), K(*this)); } - return ret; } @@ -286,9 +428,8 @@ int ObTxCallbackList::reverse_search_callback_by_seq_no(const transaction::ObTxS } }, true/*is_reverse*/); - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(ERROR, "search callbacks wont report error", K(ret), K(functor)); } else { search_res = functor.get_search_result(); @@ -297,18 +438,79 @@ int ObTxCallbackList::reverse_search_callback_by_seq_no(const transaction::ObTxS return ret; } +// caller must has hold log_latch_ +// fill log from log_cursor -> end +__attribute__((noinline)) +int ObTxCallbackList::fill_log(ObITransCallback* log_cursor, ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &functor) +{ + // the remove callback operations is either remove logged callback or unlogged cabback + // if remove logged callback, it is not conjunct with fill_log, because fill_log only + // access callbacks which must has not bee submitted and synced + // if remove unlogged callback, like rollback_to_savepoint, clean_unlogged_callback, sync_log_fail etc + // these operation is exclusive with fill_log in TxCtx's level by using exclusive lock on TxCtx + int ret = OB_SUCCESS; + if (log_cursor == &head_) { + // NOTE: log_cursor is un-stable, can not go through next `callack_` + // which may caused some callback skipped + } else { + functor.reset(); + LockState lock_state; + ret = callback_(functor, log_cursor->get_prev(), &head_, lock_state); + ctx.helper_->max_seq_no_ = MAX(functor.get_max_seq_no(), ctx.helper_->max_seq_no_); + ctx.helper_->data_size_ += functor.get_data_size(); + ctx.callback_scope_->data_size_ += functor.get_data_size(); + } + TRANS_LOG(TRACE, "[FILL LOG] list_fill_log done", K(ret), K(log_cursor)); + return ret; +} + +int ObTxCallbackList::submit_log_succ(const ObCallbackScope &callbacks) +{ + int ret = OB_SUCCESS; + // when log submitted, update log_cursor_ point to next to-log callback + // the thread has hold Tx-ctx's FLUSH_REDO lock and callback_list's log_lock_ + // this promise update log_cursor is serialized, but the next position + // may be unstable due to append callback is allowed race with this by writer + // thread, this can only happen if this scope's end is tail + ObITransCallback *next = (*callbacks.end_)->get_next(); + if (next == &head_) { + // next is un-stable, need serialize with append + LockGuard guard(*this, LOCK_MODE::LOCK_APPEND); + ATOMIC_STORE(&log_cursor_, (*callbacks.end_)->get_next()); + } else { + ATOMIC_STORE(&log_cursor_, next); + } + ATOMIC_AAF(&logged_, (int64_t)callbacks.cnt_); + ATOMIC_AAF(&logged_data_size_, callbacks.data_size_); + return ret; +} + +int ObTxCallbackList::sync_log_succ(const share::SCN scn, int64_t sync_cnt) +{ + // because log-succ callback is concurrent with submit log + // so the sync_scn_ may be update before log_cursor updated + int ret = OB_SUCCESS; + sync_scn_.atomic_store(scn); + synced_ += sync_cnt; + return ret; +} + int ObTxCallbackList::sync_log_fail(const ObCallbackScope &callbacks, + const share::SCN scn, int64_t &removed_cnt) { int ret = OB_SUCCESS; ObSyncLogFailFunctor functor; - - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor, callbacks))) { + functor.max_committed_scn_ = scn; + // prevent append, because table-lock will not acquire tx-ctx'lock to append callback + // the sync_log_fail will acquire tx-ctx' whole lock, which prevent other operations + // on tx-ctx like : fast-commit, fill_redo, backfill-log-scn, remove-callbacks etc. + // hence hold append_latch is enough + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (OB_FAIL(callback_(functor, callbacks, guard.state_))) { TRANS_LOG(WARN, "clean unlog callbacks failed", K(ret), K(functor)); } else { - TRANS_LOG(INFO, "sync failed log successfully", K(functor), K(*this)); + TRANS_LOG(INFO, "handle sync log fail success", K(functor), K(*this)); } removed_cnt = functor.get_remove_cnt(); return ret; @@ -319,9 +521,10 @@ int ObTxCallbackList::clean_unlog_callbacks(int64_t &removed_cnt) int ret = OB_SUCCESS; ObCleanUnlogCallbackFunctor functor; - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (log_cursor_ == &head_) { + // empty set, all were logged + } else if (OB_FAIL(callback_(functor, log_cursor_->get_prev(), &head_, guard.state_))) { TRANS_LOG(WARN, "clean unlog callbacks failed", K(ret), K(functor)); } else { TRANS_LOG(INFO, "clean unlog callbacks", K(functor), K(*this)); @@ -335,9 +538,9 @@ int ObTxCallbackList::get_memtable_key_arr_w_timeout(transaction::ObMemtableKeyA int ret = OB_SUCCESS; ObGetMemtableKeyWTimeoutFunctor functor(memtable_key_arr); - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + ObTimeGuard tg("get memtable key arr", 500 * 1000L); + LockGuard guard(*this, LOCK_MODE::LOCK_ALL, &tg); + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(WARN, "get memtable key arr failed", K(ret), K(functor)); } @@ -347,16 +550,16 @@ int ObTxCallbackList::get_memtable_key_arr_w_timeout(transaction::ObMemtableKeyA int ObTxCallbackList::tx_calc_checksum_before_scn(const SCN scn) { int ret = OB_SUCCESS; - ObByteLockGuard guard(latch_); - - ObCalcChecksumFunctor functor(scn); + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); + const share::SCN stop_scn = scn.is_max() ? sync_scn_ : scn; + ObCalcChecksumFunctor functor(stop_scn); functor.set_checksumer(checksum_scn_, &batch_checksum_); - if (OB_FAIL(callback_(functor))) { + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(ERROR, "calc checksum failed", K(ret)); } else { ensure_checksum_(functor.get_checksum_last_scn()); - TRANS_LOG(INFO, "calc checksum before log ts", K(functor), K(*this)); + TRANS_LOG(INFO, "calc checksum before log ts", K(scn), K(stop_scn), K(functor), KPC(this)); } return ret; @@ -366,17 +569,19 @@ int ObTxCallbackList::tx_calc_checksum_before_scn(const SCN scn) int ObTxCallbackList::tx_calc_checksum_all() { int ret = OB_SUCCESS; - ObByteLockGuard guard(latch_); - - ObCalcChecksumFunctor functor; - functor.set_checksumer(checksum_scn_, &batch_checksum_); - - if (OB_FAIL(callback_(functor))) { - TRANS_LOG(ERROR, "calc checksum wont report error", K(ret), K(functor)); + if (OB_UNLIKELY(checksum_scn_.is_max())) { + // skip repeate calc checksum } else { - ensure_checksum_(SCN::max_scn()); - } + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + ObCalcChecksumFunctor functor; + functor.set_checksumer(checksum_scn_, &batch_checksum_); + if (OB_FAIL(callback_(functor, guard.state_))) { + TRANS_LOG(ERROR, "calc checksum wont report error", K(ret), K(functor)); + } else { + ensure_checksum_(SCN::max_scn()); + } + } return ret; } @@ -384,11 +589,12 @@ int ObTxCallbackList::tx_commit() { int ret = OB_SUCCESS; ObTxEndFunctor functor(true/*is_commit*/); - - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + // exclusive with fast_commit, because fast-commit maybe in-progress + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(WARN, "trans commit failed", K(ret), K(functor)); + } else if (length_ != 0) { + ob_abort(); } else { callback_mgr_.add_tx_end_callback_remove_cnt(functor.get_remove_cnt()); } @@ -400,11 +606,12 @@ int ObTxCallbackList::tx_abort() { int ret = OB_SUCCESS; ObTxEndFunctor functor(false/*is_commit*/); - - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + // exclusive with fast_commit, because fast-commit maybe in-progress + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(WARN, "trans abort failed", K(ret), K(functor)); + } else if (length_ != 0) { + ob_abort(); } else { callback_mgr_.add_tx_end_callback_remove_cnt(functor.get_remove_cnt()); } @@ -419,10 +626,8 @@ int ObTxCallbackList::tx_elr_preparing() [](ObITransCallback *callback) -> int { return callback->elr_trans_preparing(); }); - - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(WARN, "trans elr preparing failed", K(ret), K(functor)); } @@ -436,69 +641,87 @@ int ObTxCallbackList::tx_print_callback() [](ObITransCallback *callback) -> int { return callback->print_callback(); }); - - ObByteLockGuard guard(latch_); - - if (OB_FAIL(callback_(functor))) { + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(WARN, "trans commit failed", K(ret), K(functor)); } return ret; } -int ObTxCallbackList::replay_fail(const SCN scn) +int ObTxCallbackList::replay_succ(const SCN scn) +{ + sync_scn_.inc_update(scn); + return OB_SUCCESS; +} + +int ObTxCallbackList::replay_fail(const SCN scn, const bool serial_replay) { int ret = OB_SUCCESS; - ObRemoveSyncCallbacksWCondFunctor functor( - // condition for remove - [scn](ObITransCallback *callback) -> bool { - if (scn == callback->get_scn()) { - return true; - } else { - return false; - } - }, // condition for stop - [scn](ObITransCallback *callback) -> bool { - if (scn != callback->get_scn()) { - return true; - } else { - return false; - } - }, - true, /*need_remove_data*/ - true /*is_reverse*/); + struct Functor final : public ObRemoveSyncCallbacksWCondFunctor { + Functor(const bool need_remove_data = true, const bool is_reverse = false) + : ObRemoveSyncCallbacksWCondFunctor(need_remove_data, is_reverse) {} + bool cond_for_remove(ObITransCallback *callback) { + return scn_ == callback->get_scn(); + } + bool cond_for_stop(ObITransCallback *callback) const { + return scn_ != callback->get_scn(); + } + share::SCN scn_; + } functor(true, true); - ObByteLockGuard guard(latch_); + functor.scn_ = scn; - if (OB_FAIL(callback_(functor))) { + LockGuard guard(*this, LOCK_MODE::LOCK_ALL); + // for replay fail of serial log, if parallel replay has happened, + // must reverse traversal from parallel_start_pos_.prev_ + ObITransCallback *start_pos = (serial_replay && parallel_start_pos_) ? parallel_start_pos_ : get_guard(); + ObITransCallback *end_pos = get_guard(); + if (OB_FAIL(callback_(functor, start_pos, end_pos, guard.state_))) { TRANS_LOG(ERROR, "replay fail failed", K(ret), K(functor)); } else { - TRANS_LOG(INFO, "replay callbacks failed and remove callbacks succeed", + TRANS_LOG(INFO, "replay log failed, revert its callbacks done", + KP(start_pos), KP(end_pos), K(functor), K(*this), K(scn)); } - + // revert counters when replay fail + ATOMIC_SAF(&appended_, functor.get_remove_cnt()); + ATOMIC_SAF(&removed_, functor.get_remove_cnt()); + ATOMIC_SAF(&logged_, functor.get_remove_cnt()); + ATOMIC_SAF(&synced_, functor.get_remove_cnt()); return OB_SUCCESS; } void ObTxCallbackList::get_checksum_and_scn(uint64_t &checksum, SCN &checksum_scn) { - ObByteLockGuard guard(latch_); - checksum = batch_checksum_.calc(); - checksum_scn = checksum_scn_; + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); + if (checksum_scn_.is_max()) { + checksum = checksum_; + checksum_scn = checksum_scn_; + } else { + checksum = batch_checksum_.calc(); + checksum_scn = checksum_scn_; + } TRANS_LOG(INFO, "get checksum and checksum_scn", KPC(this), K(checksum), K(checksum_scn)); } void ObTxCallbackList::update_checksum(const uint64_t checksum, const SCN checksum_scn) { - ObByteLockGuard guard(latch_); + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); batch_checksum_.set_base(checksum); checksum_scn_.atomic_set(checksum_scn); + if (checksum_scn.is_max()) { + checksum_ = checksum; + } TRANS_LOG(INFO, "update checksum and checksum_scn", KPC(this), K(checksum), K(checksum_scn)); } void ObTxCallbackList::ensure_checksum_(const SCN scn) { - if (SCN::min_scn() == scn) { + if (is_skip_checksum_()) { + } else if (!scn.is_valid()) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "checksum scn is invalid", KPC(this)); + } else if (SCN::min_scn() == scn) { // Case1: no callback is invovled } else if (scn < checksum_scn_) { // Case2: no checksum is calculated @@ -519,15 +742,146 @@ transaction::ObPartTransCtx *ObTxCallbackList::get_trans_ctx() const DEF_TO_STRING(ObTxCallbackList) { int64_t pos = 0; + transaction::ObPartTransCtx *tx_ctx = get_trans_ctx(); + const transaction::ObTransID tx_id = tx_ctx ? tx_ctx->get_trans_id() : transaction::ObTransID(); + const share::ObLSID ls_id = tx_ctx ? tx_ctx->get_ls_id() : ObLSID(); J_OBJ_START(); - J_KV(KPC(get_trans_ctx()), + J_KV(K_(id), + KP(tx_ctx), + K(tx_id), + K(ls_id), + K_(appended), K_(length), + K_(logged), + K_(removed), + K_(branch_removed), + K_(unlog_removed), + K_(sync_scn), + KP_(parallel_start_pos), + "parallel_start_scn", (parallel_start_pos_ ? parallel_start_pos_->get_scn() : share::SCN::invalid_scn()), + "skip_checksum", is_skip_checksum_(), K_(checksum_scn), K_(checksum), - K_(tmp_checksum)); + K_(tmp_checksum), + K_(batch_checksum)); J_OBJ_END(); return pos; } +int ObTxCallbackList::get_stat_for_display(ObTxCallbackListStat &stat) const +{ + int ret = OB_SUCCESS; +#define _ASSIGN_STAT_(x) stat.x = x; + LST_DO(_ASSIGN_STAT_, (), id_, sync_scn_, checksum_scn_, length_, logged_, removed_); +#undef __ASSIGN_STAT_ + return ret; +} + +bool ObTxCallbackList::find(ObITxCallbackFinder &func) +{ + int ret = OB_SUCCESS; + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); + if (OB_FAIL(callback_(func, guard.state_))) { + TRANS_LOG(WARN, "", K(ret)); + } + return func.is_found(); +} + +bool ObTxCallbackList::check_all_redo_flushed(const bool quite) const +{ + bool ok = (log_cursor_ == &head_) && + (appended_ == (logged_ + unlog_removed_)); + if (!ok && !quite) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "callback list is not all flushed", KPC(this)); + } + return ok; +} + +__attribute__((noinline)) +int64_t ObTxCallbackList::get_log_epoch() const +{ + return log_cursor_ == &head_ ? INT64_MAX : log_cursor_->get_epoch(); +} + +void ObTxCallbackList::inc_update_sync_scn(const share::SCN scn) +{ + sync_scn_.inc_update(scn); +} + +inline bool ObTxCallbackList::is_skip_checksum_() const +{ + return callback_mgr_.skip_checksum_calc(); +} + +inline bool ObTxCallbackList::is_append_only_() const +{ + return callback_mgr_.is_callback_list_append_only(id_); +} + +ObTxCallbackList::LockGuard::LockGuard(ObTxCallbackList &list, + const ObTxCallbackList::LOCK_MODE mode, + ObTimeGuard *tg) + : host_(list) +{ + using LOCK_MODE = ObTxCallbackList::LOCK_MODE; + switch(mode) { + case LOCK_MODE::LOCK_ALL: + lock_iterate_(false); + if (tg) { + tg->click(); + } + lock_append_(); + break; + case LOCK_MODE::LOCK_APPEND: + lock_append_(); + break; + case LOCK_MODE::LOCK_ITERATE: + lock_iterate_(false); + break; + case LOCK_MODE::TRY_LOCK_ITERATE: + lock_iterate_(true); + break; + default: + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "invalid lock mode", K(mode)); + } +} + +void ObTxCallbackList::LockGuard::lock_iterate_(const bool try_lock) +{ + if (!state_.ITERATE_LOCKED_) { + if (try_lock) { + if (host_.iter_synced_latch_.try_lock()) { + state_.ITERATE_LOCKED_ = true; + } + } else { + host_.iter_synced_latch_.lock(); + state_.ITERATE_LOCKED_ = true; + } + if (state_.ITERATE_LOCKED_) { + if (!host_.is_append_only_()) { + lock_append_(); + } + } + } +} + +void ObTxCallbackList::LockGuard::lock_append_() +{ + if (!state_.APPEND_LOCKED_) { + host_.append_latch_.lock(); + state_.APPEND_LOCKED_ = true; + } +} + +ObTxCallbackList::LockGuard::~LockGuard() +{ + if (state_.APPEND_LOCKED_) { + host_.append_latch_.unlock(); + } + if (state_.ITERATE_LOCKED_) { + host_.iter_synced_latch_.unlock(); + } +} + } // memtable } // oceanbase diff --git a/src/storage/memtable/mvcc/ob_tx_callback_list.h b/src/storage/memtable/mvcc/ob_tx_callback_list.h index d98d980598..eac8c195c3 100644 --- a/src/storage/memtable/mvcc/ob_tx_callback_list.h +++ b/src/storage/memtable/mvcc/ob_tx_callback_list.h @@ -15,7 +15,6 @@ #include "storage/memtable/mvcc/ob_tx_callback_functor.h" #include "storage/memtable/ob_memtable_util.h" - namespace oceanbase { namespace memtable @@ -23,16 +22,20 @@ namespace memtable class ObTransCallbackMgr; class ObCallbackScope; +class ObTxCallbackListStat; class ObTxCallbackList { public: - ObTxCallbackList(ObTransCallbackMgr &callback_mgr); + ObTxCallbackList(ObTransCallbackMgr &callback_mgr, const int16_t id); ~ObTxCallbackList(); void reset(); // append_callback will append your callback into the callback list - int append_callback(ObITransCallback *callback, const bool for_replay); + int append_callback(ObITransCallback *callback, + const bool for_replay, + const bool parallel_replay = false, + const bool serial_final = false); // concat_callbacks will append all callbacks in other into itself and reset // other. And it will return the concat number during concat_callbacks. @@ -42,10 +45,8 @@ public: // parameter _fast_commit_callback_count. It will only remove callbacks // without removing data by calling checkpoint_callback. So user need // implement lazy callback for the correctness. What's more, it will calculate - // checksum when removing. Finally it returns meet_generate_cursor if you remove - // the callbacks that generate_cursor is pointing to. - int remove_callbacks_for_fast_commit(const ObITransCallback *generate_cursor, - bool &meet_generate_cursor); + // checksum when removing. + int remove_callbacks_for_fast_commit(const share::SCN stop_scn = share::SCN::invalid_scn()); // remove_callbacks_for_remove_memtable will remove all callbacks that is // belonged to the specified memtable sets. It will only remove callbacks @@ -55,13 +56,15 @@ public: // What's more, it will calculate checksum when removing. int remove_callbacks_for_remove_memtable( const memtable::ObMemtableSet *memtable_set, - const share::SCN max_applied_scn); + const share::SCN stop_scn = share::SCN::invalid_scn()); // remove_callbacks_for_rollback_to will remove callbacks from back to front // until callbacks smaller or equal than the seq_no. It will remove both // callbacks and data by calling rollback_callback. For synced callback we need // calculate checksum and for unsynced one we need remove them. - int remove_callbacks_for_rollback_to(const transaction::ObTxSEQ to_seq_no); + int remove_callbacks_for_rollback_to(const transaction::ObTxSEQ to_seq, + const transaction::ObTxSEQ from_seq, + const share::SCN replay_scn); // reverse_search_callback_by_seq_no will search callback from back to front // until callbacks smaller or equal than the seq_no @@ -73,10 +76,12 @@ public: // clean_unlog_callbacks will remove all unlogged callbacks. Which is called // when switch to follower forcely. int clean_unlog_callbacks(int64_t &removed_cnt); - + int fill_log(ObITransCallback* log_cursor, ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &functor); + int submit_log_succ(const ObCallbackScope &callbacks); + int sync_log_succ(const share::SCN scn, int64_t sync_cnt); // sync_log_fail will remove all callbacks that not sync successfully. Which // is called when callback is on failure. - int sync_log_fail(const ObCallbackScope &callbacks, int64_t &removed_cnt); + int sync_log_fail(const ObCallbackScope &callbacks, const share::SCN scn, int64_t &removed_cnt); // tx_calc_checksum_before_scn will calculate checksum during execution. It will // remember the intermediate results for final result. @@ -103,37 +108,109 @@ public: // tx_print_callback will simply print all calbacks. int tx_print_callback(); + // dump stat info to buffer for display + int get_stat_for_display(ObTxCallbackListStat &stat) const; + + // when replay_succ, advance sync_scn, allow fast commit and calc checksum + int replay_succ(const share::SCN scn); + // replay_fail will rollback all redo in a single log according to // scn - int replay_fail(const share::SCN scn); + int replay_fail(const share::SCN scn, const bool serial_replay); + // traversal to find and break + bool find(ObITxCallbackFinder &func); private: - int callback_(ObITxCallbackFunctor &func); + union LockState { + LockState() : v_(0) {} + uint8_t v_; + struct { + bool APPEND_LOCKED_: 1; + bool ITERATE_LOCKED_: 1; + }; + bool is_locked() const { return v_ != 0; } + }; + enum class LOCK_MODE { + LOCK_ITERATE = 1, + LOCK_APPEND = 2, + LOCK_ALL = 3, + TRY_LOCK_ITERATE = 4 + }; + struct LockGuard { + LockGuard(ObTxCallbackList &host, const LOCK_MODE m, ObTimeGuard *tg = NULL); + ~LockGuard(); + bool is_locked() const { return state_.is_locked(); } + union LockState state_; + ObTxCallbackList &host_; + private: + void lock_append_(); + void lock_iterate_(const bool try_lock); + }; + friend class LockGuard; + bool is_append_only_() const; +private: + int callback_(ObITxCallbackFunctor &func, + const LockState lock_state); int callback_(ObITxCallbackFunctor &functor, - const ObCallbackScope &callbacks); + const ObCallbackScope &callbacks, + const LockState lock_state); int callback_(ObITxCallbackFunctor &func, ObITransCallback *start, - ObITransCallback *end); + ObITransCallback *end, + const LockState lock_state); int64_t calc_need_remove_count_for_fast_commit_(); void ensure_checksum_(const share::SCN scn); + bool is_skip_checksum_() const; public: ObITransCallback *get_guard() { return &head_; } ObITransCallback *get_tail() { return head_.get_prev(); } + ObITransCallback *get_log_cursor() const { return log_cursor_; } + int64_t get_log_epoch() const; + share::SCN get_sync_scn() const { return sync_scn_; } + bool check_all_redo_flushed(const bool quite = true) const; + common::ObByteLock *try_lock_log() + { + return log_latch_.try_lock() ? &log_latch_ : NULL; + } bool empty() const { return head_.get_next() == &head_; } + int64_t get_appended() const { return appended_; } int64_t get_length() const { return length_; } - int64_t get_checksum() const { return checksum_; } + int64_t get_logged() const { return logged_; } + int64_t get_synced() const { return synced_; } + int64_t get_removed() const { return removed_; } + int64_t get_unlog_removed() const { return unlog_removed_; } + int64_t get_branch_removed() const { return branch_removed_; } + uint64_t get_checksum() const { return checksum_; } int64_t get_tmp_checksum() const { return tmp_checksum_; } share::SCN get_checksum_scn() const { return checksum_scn_; } void get_checksum_and_scn(uint64_t &checksum, share::SCN &checksum_scn); void update_checksum(const uint64_t checksum, const share::SCN checksum_scn); + void inc_update_sync_scn(const share::SCN scn); transaction::ObPartTransCtx *get_trans_ctx() const; - + bool pending_log_too_large(const int64_t limit) const + { + return ATOMIC_LOAD(&data_size_) - ATOMIC_LOAD(&logged_data_size_) > limit; + } DECLARE_TO_STRING; - private: + const int16_t id_; // callback list sentinel ObITransCallback head_; + ObITransCallback *log_cursor_; + ObITransCallback *parallel_start_pos_; int64_t length_; + // stats + int64_t appended_; + int64_t logged_; + int64_t synced_; + int64_t removed_; + int64_t unlog_removed_; + int64_t branch_removed_; + int64_t data_size_; + int64_t logged_data_size_; + // the max scn of synced callback + // fast commit remove callback should not cross this + share::SCN sync_scn_; /* * transaction checksum calculation * A checksum would be calculated when a transaction commits to ensure data consistency. @@ -151,13 +228,17 @@ private: * Notice, apart from 1, the other two calculation should increment * checksum_scn_ by 1 to avoid duplicate calculation. */ - common::ObBatchChecksum batch_checksum_; + TxChecksum batch_checksum_; share::SCN checksum_scn_; uint64_t checksum_; uint64_t tmp_checksum_; - ObTransCallbackMgr &callback_mgr_; - common::ObByteLock latch_; + // used to serialize append callback to list tail + common::ObByteLock append_latch_; + // used to serialize fill and flush log of this list + common::ObByteLock log_latch_; + // used to serialize operates on synced callbacks + common::ObByteLock iter_synced_latch_; DISALLOW_COPY_AND_ASSIGN(ObTxCallbackList); }; diff --git a/src/storage/memtable/ob_memtable.cpp b/src/storage/memtable/ob_memtable.cpp index 6bd229f023..6e3dd6279b 100644 --- a/src/storage/memtable/ob_memtable.cpp +++ b/src/storage/memtable/ob_memtable.cpp @@ -114,12 +114,11 @@ ObMemtable::ObMemtable() max_data_schema_version_(0), pending_cb_cnt_(0), unsubmitted_cnt_(0), - unsynced_cnt_(0), memtable_mgr_op_cnt_(0), logging_blocked_(false), logging_blocked_start_time(0), unset_active_memtable_logging_blocked_(false), - resolve_active_memtable_left_boundary_(true), + resolved_active_memtable_left_boundary_(true), transfer_freeze_flag_(false), recommend_snapshot_version_(share::SCN::invalid_scn()), freeze_scn_(SCN::max_scn()), @@ -265,12 +264,11 @@ void ObMemtable::destroy() state_ = ObMemtableState::INVALID; freeze_state_ = ObMemtableFreezeState::INVALID; unsubmitted_cnt_ = 0; - unsynced_cnt_ = 0; memtable_mgr_op_cnt_ = 0; logging_blocked_ = false; logging_blocked_start_time = 0; unset_active_memtable_logging_blocked_ = false; - resolve_active_memtable_left_boundary_ = true; + resolved_active_memtable_left_boundary_ = true; transfer_freeze_flag_ = false; recommend_snapshot_version_.reset(); max_end_scn_ = ObScnRange::MIN_SCN; @@ -292,11 +290,10 @@ int ObMemtable::safe_to_destroy(bool &is_safe) int64_t ref_cnt = get_ref(); int64_t write_ref_cnt = get_write_ref(); int64_t unsubmitted_cnt = get_unsubmitted_cnt(); - int64_t unsynced_cnt = get_unsynced_cnt(); is_safe = (0 == ref_cnt && 0 == write_ref_cnt); if (is_safe) { - is_safe = (0 == unsubmitted_cnt && 0 == unsynced_cnt); + is_safe = (0 == unsubmitted_cnt); } return ret; @@ -662,7 +659,6 @@ int ObMemtable::exist( { int ret = OB_SUCCESS; ObMemtableKey parameter_mtk; - ObMemtableKey returned_mtk; ObMvccValueIterator value_iter; ObQueryFlag query_flag; ObStoreRowLockState lock_state; @@ -683,7 +679,7 @@ int ObMemtable::exist( } else if (OB_FAIL(mvcc_engine_.get(context.store_ctx_->mvcc_acc_ctx_, query_flag, ¶meter_mtk, - &returned_mtk, + NULL, value_iter, lock_state))) { TRANS_LOG(WARN, "get value iter fail, ", K(ret)); @@ -1120,6 +1116,7 @@ int ObMemtable::replay_schema_version_change_log(const int64_t schema_version) } int ObMemtable::replay_row(ObStoreCtx &ctx, + const share::SCN &scn, ObMemtableMutatorIterator *mmi) { int ret = OB_SUCCESS; @@ -1138,8 +1135,6 @@ int ObMemtable::replay_row(ObStoreCtx &ctx, blocksstable::ObDmlFlag dml_flag = blocksstable::ObDmlFlag::DF_NOT_EXIST; ObMemtableCtx *mt_ctx = ctx.mvcc_acc_ctx_.mem_ctx_; ObPartTransCtx *part_ctx = static_cast(mt_ctx->get_trans_ctx()); - const SCN scn = mt_ctx->get_redo_scn(); - const int64_t log_id = mt_ctx->get_redo_log_id(); common::ObTimeGuard timeguard("ObMemtable::replay_row", 5 * 1000); if (OB_FAIL(mmi->get_mutator_row().copy(table_id, rowkey, table_version, row, @@ -1160,7 +1155,9 @@ int ObMemtable::replay_row(ObStoreCtx &ctx, lib::CompatModeGuard compat_guard(mode_); ObMemtableData mtd(dml_flag, row.size_, row.data_); ObMemtableKey mtk; - ObTxNodeArg arg(&mtd, /*memtable_data*/ + const transaction::ObTransID tx_id = ctx.mvcc_acc_ctx_.tx_id_; + ObTxNodeArg arg(tx_id, /*trans id*/ + &mtd, /*memtable_data*/ NULL, /*old_row*/ version, /*memstore_version*/ seq_no, /*seq_no*/ @@ -1182,9 +1179,7 @@ int ObMemtable::replay_row(ObStoreCtx &ctx, K(modify_count), K(acc_checksum)); } } else { - if (part_ctx->need_update_schema_version(log_id, scn)) { - ctx.mvcc_acc_ctx_.mem_ctx_->set_table_version(table_version); - } + ctx.mvcc_acc_ctx_.mem_ctx_->set_table_version(table_version); if (dml_flag != blocksstable::ObDmlFlag::DF_LOCK) { set_max_data_schema_version(table_version); set_max_column_cnt(column_cnt); @@ -1674,7 +1669,7 @@ int ObMemtable::dec_unsubmitted_cnt() // must maintain the order of getting variables to avoid concurrency problems // is_frozen_memtable() can affect wirte_ref_cnt - // write_ref_cnt can affect unsubmitted_cnt and unsynced_cnt + // write_ref_cnt can affect unsubmitted_cnt bool is_frozen = is_frozen_memtable(); int64_t write_ref_cnt = get_write_ref(); int64_t new_unsubmitted_cnt = get_unsubmitted_cnt(); @@ -1712,7 +1707,7 @@ int64_t ObMemtable::dec_write_ref() // must maintain the order of getting variables to avoid concurrency problems // is_frozen_memtable() can affect wirte_ref_cnt - // write_ref_cnt can affect unsubmitted_cnt and unsynced_cnt + // write_ref_cnt can affect unsubmitted_cnt bool is_frozen = is_frozen_memtable(); int64_t new_write_ref_cnt = get_write_ref(); int64_t unsubmitted_cnt = get_unsubmitted_cnt(); @@ -1720,49 +1715,20 @@ int64_t ObMemtable::dec_write_ref() 0 == new_write_ref_cnt && 0 == unsubmitted_cnt) { (void)unset_logging_blocked_for_active_memtable(); - if (0 == get_unsynced_cnt()) { + share::SCN right_boundary; + if (OB_FAIL(get_ls_current_right_boundary(right_boundary))) { + TRANS_LOG(WARN, "get ls right bound fail", K(ret), K(ls_id), KPC(this)); + } else if (right_boundary >= get_max_end_scn()) { resolve_right_boundary(); - (void)resolve_left_boundary_for_active_memtable(); + if (OB_LIKELY(!get_resolved_active_memtable_left_boundary())) { + resolve_left_boundary_for_active_memtable(); + } } } return old_write_ref_cnt; } -void ObMemtable::inc_unsynced_cnt() -{ - int64_t unsynced_cnt = inc_unsynced_cnt_(); - TRANS_LOG(DEBUG, "inc_unsynced_cnt", K(ls_id_), K(unsynced_cnt), KPC(this), K(lbt())); -} - -int ObMemtable::dec_unsynced_cnt() -{ - int ret = OB_SUCCESS; - share::ObLSID ls_id = freezer_->get_ls_id(); - - int64_t old_unsynced_cnt = dec_unsynced_cnt_(); - - // must maintain the order of getting variables to avoid concurrency problems - // is_frozen_memtable() can affect wirte_ref_cnt - // write_ref_cnt can affect unsubmitted_cnt and unsynced_cnt - bool is_frozen = is_frozen_memtable(); - int64_t write_ref_cnt = get_write_ref(); - int64_t new_unsynced_cnt = get_unsynced_cnt(); - TRANS_LOG(DEBUG, "dec_unsynced_cnt", K(ls_id), KPC(this), K(lbt())); - if (OB_UNLIKELY(old_unsynced_cnt < 0)) { - TRANS_LOG(ERROR, "unsynced_cnt not match", K(ret), K(ls_id), KPC(this)); - } else if (is_frozen && - 0 == write_ref_cnt && - 0 == new_unsynced_cnt) { - resolve_right_boundary(); - TRANS_LOG(INFO, "[resolve_right_boundary] dec_unsynced_cnt", K(ls_id), KPC(this)); - (void)resolve_left_boundary_for_active_memtable(); - TRANS_LOG(INFO, "memtable log synced", K(ret), K(ls_id), KPC(this)); - } - - return ret; -} - void ObMemtable::unset_logging_blocked_for_active_memtable() { int ret = OB_SUCCESS; @@ -1838,28 +1804,6 @@ int64_t ObMemtable::dec_unsubmitted_cnt_() return ATOMIC_SAF(&unsubmitted_cnt_, 1); } -int64_t ObMemtable::inc_unsynced_cnt_() -{ - return ATOMIC_AAF(&unsynced_cnt_, 1); -} - -int64_t ObMemtable::dec_unsynced_cnt_() -{ - return ATOMIC_SAF(&unsynced_cnt_, 1); -} - -void ObMemtable::inc_unsubmitted_and_unsynced_cnt() -{ - inc_unsubmitted_cnt(); - inc_unsynced_cnt(); -} - -void ObMemtable::dec_unsubmitted_and_unsynced_cnt() -{ - dec_unsubmitted_cnt(); - dec_unsynced_cnt(); -} - bool ObMemtable::can_be_minor_merged() { return is_in_prepare_list_of_data_checkpoint(); @@ -1988,7 +1932,7 @@ int ObMemtable::set_end_scn(const SCN freeze_scn) return ret; } -int ObMemtable::set_max_end_scn(const SCN scn) +int ObMemtable::set_max_end_scn(const SCN scn, bool allow_backoff) { int ret = OB_SUCCESS; share::ObLSID ls_id = freezer_->get_ls_id(); @@ -2003,6 +1947,11 @@ int ObMemtable::set_max_end_scn(const SCN scn) ret = OB_SCN_OUT_OF_BOUND; TRANS_LOG(WARN, "cannot set max end log ts smaller to start log ts", K(ret), K(scn), K(ls_id), KPC(this)); + } else if (allow_backoff) { + TRANS_LOG(INFO, "set max_end_scn force", K(scn), K(max_end_scn_.atomic_get()), K(key_), KPC(this)); + if (scn != max_end_scn_.atomic_get()) { + max_end_scn_.dec_update(scn); + } } else { SCN old_max_end_scn; SCN new_max_end_scn = get_max_end_scn(); @@ -2047,7 +1996,7 @@ bool ObMemtable::rec_scn_is_stable() int ret = OB_SUCCESS; bool rec_scn_is_stable = false; if (SCN::max_scn() == rec_scn_) { - rec_scn_is_stable = (is_frozen_memtable() && write_ref_cnt_ == 0 && unsynced_cnt_ == 0); + rec_scn_is_stable = (is_frozen_memtable() && write_ref_cnt_ == 0 && unsubmitted_cnt_ == 0); } else { SCN max_consequent_callbacked_scn; if (OB_FAIL(freezer_->get_max_consequent_callbacked_scn(max_consequent_callbacked_scn))) { @@ -2075,7 +2024,7 @@ bool ObMemtable::rec_scn_is_stable() return rec_scn_is_stable; } -int ObMemtable::get_current_right_boundary(SCN ¤t_right_boundary) +int ObMemtable::get_ls_current_right_boundary(SCN ¤t_right_boundary) { int ret = OB_SUCCESS; if (OB_ISNULL(freezer_)) { @@ -2104,8 +2053,8 @@ bool ObMemtable::ready_for_flush_() { bool is_frozen = is_frozen_memtable(); int64_t write_ref_cnt = get_write_ref(); - int64_t unsynced_cnt = get_unsynced_cnt(); - bool bool_ret = is_frozen && 0 == write_ref_cnt && 0 == unsynced_cnt; + int64_t unsubmitted_cnt = get_unsubmitted_cnt(); + bool bool_ret = is_frozen && 0 == write_ref_cnt && 0 == unsubmitted_cnt; int ret = OB_SUCCESS; SCN current_right_boundary = ObScnRange::MIN_SCN; @@ -2116,13 +2065,17 @@ bool ObMemtable::ready_for_flush_() } else if (OB_FAIL(resolve_max_end_scn_())) { TRANS_LOG(WARN, "fail to resolve max_end_scn", K(ret), KPC(this), K(ls_id)); } else { - resolve_right_boundary(); TRANS_LOG(INFO, "[resolve_right_boundary] ready_for_flush_", K(ls_id), KPC(this)); - if (OB_FAIL(get_current_right_boundary(current_right_boundary))) { + if (OB_FAIL(get_ls_current_right_boundary(current_right_boundary))) { TRANS_LOG(WARN, "fail to get current right boundary", K(ret)); } - bool_ret = current_right_boundary >= get_end_scn() && - (is_empty() || get_resolve_active_memtable_left_boundary()); + if ((bool_ret = (current_right_boundary >= get_max_end_scn()))) { + resolve_right_boundary(); + if (!get_resolved_active_memtable_left_boundary()) { + resolve_left_boundary_for_active_memtable(); + } + bool_ret = (is_empty() || get_resolved_active_memtable_left_boundary()); + } if (bool_ret) { freeze_state_ = ObMemtableFreezeState::READY_FOR_FLUSH; if (0 == mt_stat_.ready_for_flush_time_) { @@ -2164,14 +2117,13 @@ bool ObMemtable::ready_for_flush_() get_end_scn(), get_write_ref(), get_unsubmitted_cnt(), - get_unsynced_cnt(), current_right_boundary.get_val_for_tx()); int tmp_ret = OB_SUCCESS; if (OB_TMP_FAIL(ADD_SUSPECT_INFO(MINI_MERGE, ObDiagnoseTabletType::TYPE_MINI_MERGE, ls_id, get_tablet_id(), ObSuspectInfoType::SUSPECT_NOT_READY_FOR_FLUSH, - static_cast(is_frozen_memtable()), get_write_ref(), get_unsynced_cnt(), + static_cast(is_frozen_memtable()), get_write_ref(), get_unsubmitted_cnt(), current_right_boundary.get_val_for_tx(), get_end_scn().get_val_for_tx()))) { STORAGE_LOG(WARN, "failed to add suspcet info", K(tmp_ret)); } @@ -2187,23 +2139,21 @@ void ObMemtable::print_ready_for_flush() const common::ObTabletID tablet_id = key_.tablet_id_; bool frozen_memtable_flag = is_frozen_memtable(); int64_t write_ref = get_write_ref(); - int64_t unsynced_cnt = get_unsynced_cnt(); SCN end_scn = get_end_scn(); SCN current_right_boundary; uint32_t logstream_freeze_clock = freezer_->get_freeze_clock(); uint32_t memtable_freeze_clock = freeze_clock_; - if (OB_FAIL(get_current_right_boundary(current_right_boundary))) { + if (OB_FAIL(get_ls_current_right_boundary(current_right_boundary))) { TRANS_LOG(WARN, "fail to get current right boundary", K(ret)); } bool bool_ret = frozen_memtable_flag && 0 == write_ref && - 0 == unsynced_cnt && current_right_boundary >= end_scn; TRANS_LOG(INFO, "[ObFreezer] print_ready_for_flush", KP(this), K(ls_id), K(tablet_id), K(ret), K(bool_ret), - K(frozen_memtable_flag), K(write_ref), K(unsynced_cnt), + K(frozen_memtable_flag), K(write_ref), K(current_right_boundary), K(end_scn), K(logstream_freeze_clock), K(memtable_freeze_clock)); } @@ -2325,6 +2275,17 @@ int ObMemtable::resolve_max_end_scn_() return ret; } +// implement NOTE: +// call this function must ensure all TxNodes on memtable has been +// logged and the log's either synced successfully or synced failed. +// because otherwise the max_end_scn is not correct and which may be larger than +// the max value of valid TxNode(s)'s log_scn, which cause an incorrect right +// boundary value, and an incorrect left boundary value of next active memtable +// +// when TxNode's log synced failed, Txn's function will process to adjust the +// max_end_scn of this memtable, finally, the memtable will has a correct right +// boundary value, keep the safety: +// future data's log_scn > max_end_scn of this memtable int ObMemtable::resolve_right_boundary() { SCN max_end_scn = get_max_end_scn(); @@ -3056,11 +3017,14 @@ int ObMemtable::set_( TRANS_LOG(ERROR, "Unexpected not exist trans node", K(ret), K(new_row)); } else { ObMemtableData mtd(new_row.flag_.get_dml_flag(), len, buf); - ObTxNodeArg arg(&mtd, /*memtable_data*/ + ObTxNodeArg arg( + ctx.mvcc_acc_ctx_.tx_id_, /*trans id*/ + &mtd, /*memtable_data*/ NULL == old_row ? NULL : &old_row_data, timestamp_, /*memstore_version*/ ctx.mvcc_acc_ctx_.tx_scn_, /*seq_no*/ - new_row.row_val_.count_ /*column_cnt*/); + new_row.row_val_.count_ /*column_cnt*/ + ); if (OB_FAIL(mvcc_write_(param, context, &mtk, arg, is_new_locked, mvcc_row, check_exist))) { if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret && @@ -3129,14 +3093,16 @@ int ObMemtable::lock_( TRANS_LOG(WARN, "Failed to writer rowkey", K(ret), K(rowkey)); } else { // for elr optimization - context.store_ctx_->mvcc_acc_ctx_.get_mem_ctx()->set_row_updated(); + ObMvccAccessCtx &acc_ctx = context.store_ctx_->mvcc_acc_ctx_; + acc_ctx.get_mem_ctx()->set_row_updated(); ObMemtableData mtd(blocksstable::ObDmlFlag::DF_LOCK, len, buf); - ObTxNodeArg arg(&mtd, /*memtable_data*/ - NULL, /*old_data*/ - timestamp_, /*memstore_version*/ - context.store_ctx_->mvcc_acc_ctx_.tx_scn_, /*seq_no*/ - rowkey.get_obj_cnt()); /*column_cnt*/ - if (context.store_ctx_->mvcc_acc_ctx_.write_flag_.is_check_row_locked()) { + ObTxNodeArg arg(acc_ctx.tx_id_, /*trans id*/ + &mtd, /*memtable_data*/ + NULL, /*old_data*/ + timestamp_, /*memstore_version*/ + acc_ctx.tx_scn_, /*seq_no*/ + rowkey.get_obj_cnt()); /*column_cnt*/ + if (acc_ctx.write_flag_.is_check_row_locked()) { if (OB_FAIL(ObRowConflictHandler::check_foreign_key_constraint(param, context, rowkey))) { if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) { TRANS_LOG(WARN, "meet unexpected return code in check_row_locked", K(ret), K(context), K(mtk)); @@ -3171,11 +3137,7 @@ int ObMemtable::mvcc_replay_(storage::ObStoreCtx &ctx, is_new_add))) { TRANS_LOG(WARN, "prepare kv before lock fail", K(ret)); } else if (FALSE_IT(timeguard.click("mvcc_engine_.create_kv"))) { - } else if (OB_FAIL(mvcc_engine_.mvcc_replay(*mem_ctx, - &stored_key, - *value, - arg, - res))) { + } else if (OB_FAIL(mvcc_engine_.mvcc_replay(arg, res))) { TRANS_LOG(WARN, "mvcc replay fail", K(ret)); } else if (FALSE_IT(timeguard.click("mvcc_engine_.mvcc_replay"))) { } else if (OB_FAIL(mvcc_engine_.ensure_kv(&stored_key, value))) { diff --git a/src/storage/memtable/ob_memtable.h b/src/storage/memtable/ob_memtable.h index 3c78238af7..5badb60b78 100644 --- a/src/storage/memtable/ob_memtable.h +++ b/src/storage/memtable/ob_memtable.h @@ -318,6 +318,7 @@ public: // decrypt_buf is used for decryption virtual int replay_row( storage::ObStoreCtx &ctx, + const share::SCN &scn, ObMemtableMutatorIterator *mmi); virtual int replay_schema_version_change_log( const int64_t schema_version); @@ -366,7 +367,7 @@ public: int64_t get_freeze_state() const { return freeze_state_; } int64_t get_protection_clock() const { return local_allocator_.get_protection_clock(); } int64_t get_retire_clock() const { return local_allocator_.get_retire_clock(); } - int get_current_right_boundary(share::SCN ¤t_right_boundary); + int get_ls_current_right_boundary(share::SCN ¤t_right_boundary); inline bool& get_read_barrier() { return read_barrier_; } inline void set_write_barrier() { write_barrier_ = true; } @@ -380,8 +381,8 @@ public: inline void set_is_flushed() { is_flushed_ = true; } inline bool get_is_flushed() { return is_flushed_; } inline void unset_active_memtable_logging_blocked() { ATOMIC_STORE(&unset_active_memtable_logging_blocked_, true); } - inline void set_resolve_active_memtable_left_boundary(bool flag) { ATOMIC_STORE(&resolve_active_memtable_left_boundary_, flag); } - inline bool get_resolve_active_memtable_left_boundary() { return ATOMIC_LOAD(&resolve_active_memtable_left_boundary_); } + inline void set_resolved_active_memtable_left_boundary(bool flag) { ATOMIC_STORE(&resolved_active_memtable_left_boundary_, flag); } + inline bool get_resolved_active_memtable_left_boundary() { return ATOMIC_LOAD(&resolved_active_memtable_left_boundary_); } void set_freeze_state(const int64_t state); void set_minor_merged(); int64_t get_minor_merged_time() const { return minor_merged_time_; } @@ -442,7 +443,7 @@ public: int set_rec_scn(share::SCN rec_scn); int set_start_scn(const share::SCN start_ts); int set_end_scn(const share::SCN freeze_ts); - int set_max_end_scn(const share::SCN scn); + int set_max_end_scn(const share::SCN scn, bool allow_backoff = false); int set_max_end_scn_to_inc_start_scn(); inline int set_logging_blocked() { @@ -472,11 +473,6 @@ public: int64_t get_unsubmitted_cnt() const { return ATOMIC_LOAD(&unsubmitted_cnt_); } int inc_unsubmitted_cnt(); int dec_unsubmitted_cnt(); - int64_t get_unsynced_cnt() { return ATOMIC_LOAD(&unsynced_cnt_); } - void inc_unsynced_cnt(); - int dec_unsynced_cnt(); - void inc_unsubmitted_and_unsynced_cnt(); - void dec_unsubmitted_and_unsynced_cnt(); virtual uint32_t get_freeze_flag() override; virtual OB_INLINE int64_t get_timestamp() const override { return timestamp_; } void inc_timestamp(const int64_t timestamp) { timestamp_ = MAX(timestamp_, timestamp + 1); } @@ -510,8 +506,8 @@ public: bool is_can_flush() { return ObMemtableFreezeState::READY_FOR_FLUSH == freeze_state_ && share::SCN::max_scn() != get_end_scn(); } INHERIT_TO_STRING_KV("ObITable", ObITable, KP(this), KP_(memtable_mgr), K_(timestamp), K_(state), K_(freeze_clock), K_(max_schema_version), K_(max_data_schema_version), K_(max_column_cnt), - K_(write_ref_cnt), K_(local_allocator), K_(unsubmitted_cnt), K_(unsynced_cnt), - K_(logging_blocked), K_(unset_active_memtable_logging_blocked), K_(resolve_active_memtable_left_boundary), + K_(write_ref_cnt), K_(local_allocator), K_(unsubmitted_cnt), + K_(logging_blocked), K_(unset_active_memtable_logging_blocked), K_(resolved_active_memtable_left_boundary), K_(contain_hotspot_row), K_(max_end_scn), K_(rec_scn), K_(snapshot_version), K_(migration_clog_checkpoint_scn), K_(is_tablet_freeze), K_(contain_hotspot_row), K_(read_barrier), K_(is_flushed), K_(freeze_state), K_(allow_freeze), @@ -619,8 +615,6 @@ private: int64_t dec_write_ref_(); int64_t inc_unsubmitted_cnt_(); int64_t dec_unsubmitted_cnt_(); - int64_t inc_unsynced_cnt_(); - int64_t dec_unsynced_cnt_(); int64_t try_split_range_for_sample_(const ObStoreRowkey &start_key, const ObStoreRowkey &end_key, const int64_t range_count, @@ -643,12 +637,11 @@ private: int64_t max_data_schema_version_; // to record the max schema version of write data int64_t pending_cb_cnt_; // number of transactions have to sync log int64_t unsubmitted_cnt_; // number of trans node to be submitted logs - int64_t unsynced_cnt_; // number of trans node to be synced logs int64_t memtable_mgr_op_cnt_; // number of operations for memtable_mgr bool logging_blocked_; // flag whether the memtable can submit log, cannot submit if true int64_t logging_blocked_start_time; // record the start time of logging blocked bool unset_active_memtable_logging_blocked_; - bool resolve_active_memtable_left_boundary_; + bool resolved_active_memtable_left_boundary_; // TODO(handora.qc): remove it as soon as possible // only used for decide special right boundary of memtable bool transfer_freeze_flag_; diff --git a/src/storage/memtable/ob_memtable_context.cpp b/src/storage/memtable/ob_memtable_context.cpp index d5417df859..5d5a904599 100644 --- a/src/storage/memtable/ob_memtable_context.cpp +++ b/src/storage/memtable/ob_memtable_context.cpp @@ -23,6 +23,7 @@ #include "storage/tx/ob_trans_ctx_mgr.h" #include "share/ob_force_print_log.h" #include "lib/utility/ob_tracepoint.h" +#include "lib/container/ob_array_helper.h" #include "storage/tx_storage/ob_ls_service.h" #include "storage/tx_table/ob_tx_table.h" #include "storage/tablelock/ob_lock_memtable.h" @@ -54,7 +55,6 @@ ObMemtableCtx::ObMemtableCtx() lock_for_read_retry_count_(0), lock_for_read_elapse_(0), trans_mem_total_size_(0), - unsynced_cnt_(0), unsubmitted_cnt_(0), callback_mem_used_(0), callback_alloc_count_(0), @@ -113,19 +113,21 @@ void ObMemtableCtx::reset() { if (IS_INIT) { if ((ATOMIC_LOAD(&callback_mem_used_) > 8 * 1024 * 1024) && REACH_TIME_INTERVAL(200000)) { - TRANS_LOG(INFO, "memtable callback used", K(*this)); + TRANS_LOG(INFO, "memtable callback memory used > 8MB", K(callback_mem_used_), K(*this)); } if (OB_UNLIKELY(callback_alloc_count_ != callback_free_count_)) { TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "callback alloc and free count not match", K(*this)); } - if (OB_UNLIKELY(unsynced_cnt_ != 0)) { - TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "txn unsynced cnt not zero", K(*this), - K(unsynced_cnt_), K(unsubmitted_cnt_)); + if (OB_UNLIKELY(unsubmitted_cnt_ != 0)) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "txn unsubmitted cnt not zero", K(*this), K(unsubmitted_cnt_)); ob_abort(); } - if (OB_UNLIKELY(unsubmitted_cnt_ != 0)) { - TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "txn unsubmitted cnt not zero", K(*this), - K(unsynced_cnt_), K(unsubmitted_cnt_)); + const int64_t fill = log_gen_.get_redo_filled_count(); + const int64_t sync_succ = log_gen_.get_redo_sync_succ_count(); + const int64_t sync_fail = log_gen_.get_redo_sync_fail_count(); + if (OB_UNLIKELY(fill != sync_succ + sync_fail)) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "redo filled_count != sync_succ + sync_fail", KPC(this), + K(fill), K(sync_succ), K(sync_fail)); ob_abort(); } is_inited_ = false; @@ -137,7 +139,6 @@ void ObMemtableCtx::reset() lock_for_read_retry_count_ = 0; lock_for_read_elapse_ = 0; truncate_cnt_ = 0; - unsynced_cnt_ = 0; unsubmitted_cnt_ = 0; lock_mem_ctx_.reset(); retry_info_.reset(); @@ -169,33 +170,19 @@ int64_t ObMemtableCtx::to_string(char *buf, const int64_t buf_len) const common::databuff_printf(buf, buf_len, pos, " end_code=%d tx_status=%ld is_readonly=%s " "ref=%ld trans_id=%s ls_id=%ld " - "callback_alloc_count=%ld callback_free_count=%ld " - "checksum=%lu tmp_checksum=%lu checksum_scn=%s " - "redo_filled_count=%ld redo_sync_succ_count=%ld " - "redo_sync_fail_count=%ld main_list_length=%ld " - "unsynced_cnt=%ld unsubmitted_cnt_=%ld " - "cb_statistics:[main=%ld, slave=%ld, merge=%ld, " - "tx_end=%ld, rollback_to=%ld, " - "fast_commit=%ld, remove_memtable=%ld]", + "row_callback[alloc:%ld, free:%ld, unsubmit:%ld] " + "redo[fill:%ld,sync_succ:%ld, sync_fail:%ld] " + "main_list_len=%ld pending_log_size=%ld ", end_code_, tx_status_, STR_BOOL(is_read_only_), ref_, NULL == ctx_ ? "" : S(ctx_->get_trans_id()), NULL == ctx_ ? -1 : ctx_->get_ls_id().id(), - callback_alloc_count_, callback_free_count_, - trans_mgr_.get_checksum(), - trans_mgr_.get_tmp_checksum(), - to_cstring(trans_mgr_.get_checksum_scn()), + callback_alloc_count_, callback_free_count_, unsubmitted_cnt_, log_gen_.get_redo_filled_count(), log_gen_.get_redo_sync_succ_count(), log_gen_.get_redo_sync_fail_count(), trans_mgr_.get_main_list_length(), - unsynced_cnt_, unsubmitted_cnt_, - trans_mgr_.get_callback_main_list_append_count(), - trans_mgr_.get_callback_slave_list_append_count(), - trans_mgr_.get_callback_slave_list_merge_count(), - trans_mgr_.get_callback_remove_for_trans_end_count(), - trans_mgr_.get_callback_remove_for_rollback_to_count(), - trans_mgr_.get_callback_remove_for_fast_commit_count(), - trans_mgr_.get_callback_remove_for_remove_memtable_count()); + trans_mgr_.get_pending_log_size()); + trans_mgr_.print_statistics(buf, buf_len, pos); common::databuff_printf(buf, buf_len, pos, "}"); return pos; } @@ -415,36 +402,37 @@ int ObMemtableCtx::trans_begin() return ret; } -int ObMemtableCtx::replay_begin(const SCN scn) +int ObMemtableCtx::replay_begin(const bool parallel_replay, const SCN scn) { - ObByteLockGuard guard(lock_); - - set_redo_scn(scn); - + // UNUSED(scn); + trans_mgr_.replay_begin(parallel_replay, scn); return OB_SUCCESS; } +// callback_list_idx: +// -1 means all callback-list should do commit/rollback int ObMemtableCtx::replay_end(const bool is_replay_succ, + const int16_t callback_list_idx, const SCN scn) { int ret = OB_SUCCESS; ObByteLockGuard guard(lock_); if (!is_replay_succ) { - ret = trans_mgr_.replay_fail(scn); + ret = trans_mgr_.replay_fail(callback_list_idx, scn); } else { - ret = trans_mgr_.replay_succ(scn); + ret = trans_mgr_.replay_succ(callback_list_idx, scn); } return ret; } -int ObMemtableCtx::rollback_redo_callbacks(const SCN scn) +int ObMemtableCtx::rollback_redo_callbacks(const int16_t callback_list_idx, const SCN scn) { int ret = OB_SUCCESS; ObByteLockGuard guard(lock_); - ret = trans_mgr_.replay_fail(scn); + ret = trans_mgr_.replay_fail(callback_list_idx, scn); return ret; } @@ -503,6 +491,7 @@ int ObMemtableCtx::do_trans_end( // and check memory leakage if (OB_UNLIKELY(ATOMIC_LOAD(&callback_alloc_count_) != ATOMIC_LOAD(&callback_free_count_))) { TRANS_LOG(ERROR, "callback alloc and free count not match", K(*this)); + ob_abort(); // for easy debug, remove later } // release durable table lock if (OB_FAIL(ret)) { @@ -553,20 +542,31 @@ int ObMemtableCtx::trans_replay_end(const bool commit, // We must calculate the checksum and generate the checksum_scn even when // the checksum verification is unnecessary. This because the trans table // merge may be triggered after clear state in which the callback has already - if (commit - && 0 != checksum + && 0 != checksum // if leader's checksum is skipped, follow skip check && log_cluster_version >= CLUSTER_VERSION_3100 && !ObServerConfig::get_instance().ignore_replay_checksum_error) { - const uint64_t checksum4 = calc_checksum_all(); - if (checksum != checksum4) { - cs_ret = OB_CHECKSUM_ERROR; - TRANS_LOG(ERROR, "MT_CTX: replay checksum error", K(ret), K(*this), - K(commit), K(checksum), K(checksum4)); + ObSEArray replay_checksum; + if (OB_FAIL(calc_checksum_all(replay_checksum))) { + TRANS_LOG(WARN, "calc checksum fail", K(ret)); + } else { + uint64_t checksum_collapsed = 0; + uint8_t _sig[replay_checksum.count()]; + ObArrayHelper checksum_signature(replay_checksum.count(), _sig); + convert_checksum_for_commit_log(replay_checksum, checksum_collapsed, checksum_signature); + if (checksum != checksum_collapsed) { + cs_ret = OB_CHECKSUM_ERROR; + TRANS_LOG(ERROR, "MT_CTX: replay checksum error", K(cs_ret), + "checksum_in_commit_log", checksum, + "checksum_replayed", checksum_collapsed, + "checksum_before_collapse", replay_checksum, + K(checksum_signature), KPC(this)); + ob_abort(); + } } } - - if (OB_FAIL(do_trans_end(commit, + if (OB_FAIL(ret)) { + } else if (OB_FAIL(do_trans_end(commit, trans_version, final_scn, commit ? OB_TRANS_COMMITED : OB_TRANS_ROLLBACKED))) { @@ -608,107 +608,74 @@ int ObMemtableCtx::commit_to_replay() return OB_SUCCESS; } -int ObMemtableCtx::fill_redo_log(char *buf, - const int64_t buf_len, - int64_t &buf_pos, - ObRedoLogSubmitHelper &helper, - const bool log_for_lock_node) +int ObMemtableCtx::fill_redo_log(ObTxFillRedoCtx &ctx) { int ret = OB_SUCCESS; - - if (NULL == buf - || 0 >= buf_len - || buf_len <= buf_pos) { - TRANS_LOG(WARN, "invalid param"); - ret = OB_INVALID_ARGUMENT; - } else { - if (OB_FAIL(log_gen_.fill_redo_log(buf, - buf_len, - buf_pos, - helper, - log_for_lock_node))) { - // When redo log data is greater than or equal to 1.875M, or participant has - // no redo log data at all, this branch would be reached. Don't print log here - } - if (OB_FAIL(ret)) { - if (buf_pos > buf_len) { - TRANS_LOG(ERROR, "unexpected buf pos", KP(buf), K(buf_len), K(buf_pos)); - ret = OB_ERR_UNEXPECTED; - } - } - } - if (OB_SUCCESS != ret && OB_EAGAIN != ret && OB_ENTRY_NOT_EXIST != ret) { - TRANS_LOG(WARN, "fill_redo_log fail", "ret", ret, "trans_id", - NULL == ctx_ ? "" : S(ctx_->get_trans_id()), - KP(buf), - "buf_len", buf_len, - "buf_pos", buf_pos, - "pending_log_size", trans_mgr_.get_pending_log_size(), - K(*this)); - } - + ret = log_gen_.fill_redo_log(ctx); return ret; } +int ObMemtableCtx::get_log_guard(const transaction::ObTxSEQ &write_seq, + ObCallbackListLogGuard &log_guard, + int& cb_list_idx) +{ + return trans_mgr_.get_log_guard(write_seq, log_guard, cb_list_idx); +} int ObMemtableCtx::log_submitted(const ObRedoLogSubmitHelper &helper) { inc_pending_log_size(-1 * helper.data_size_); inc_flushed_log_size(helper.data_size_); - return log_gen_.log_submitted(helper.callbacks_); + return log_gen_.log_submitted(helper.callbacks_, helper.log_scn_); } -int ObMemtableCtx::sync_log_succ(const SCN scn, const ObCallbackScope &callbacks) +int ObMemtableCtx::sync_log_succ(const SCN scn, const ObCallbackScopeArray &callbacks) { - int ret = OB_SUCCESS; - - if (OB_SUCCESS == ATOMIC_LOAD(&end_code_)) { - if (OB_FAIL(log_gen_.sync_log_succ(scn, callbacks))) { - TRANS_LOG(WARN, "sync log failed", K(ret)); - } - } else { - if (!callbacks.is_empty()) { - TRANS_LOG(INFO, "No memtable callbacks because of trans_end", K(end_code_), K(scn), - KPC(ctx_)); - } - } - - return ret; + lock_mem_ctx_.sync_log_succ(scn); + log_gen_.sync_log_succ(callbacks, scn); + return OB_SUCCESS; } -void ObMemtableCtx::sync_log_fail(const ObCallbackScope &callbacks) +void ObMemtableCtx::sync_log_fail(const ObCallbackScopeArray &callbacks, + const share::SCN &max_applied_scn) { int ret = OB_SUCCESS; - if (!callbacks.is_empty()) { + if (callbacks.count() > 0) { set_partial_rollbacked(); } if (OB_SUCCESS == ATOMIC_LOAD(&end_code_)) { if (OB_FAIL(reuse_log_generator_())) { TRANS_LOG(ERROR, "fail to reset log generator", K(ret)); } else { - log_gen_.sync_log_fail(callbacks); + log_gen_.sync_log_fail(callbacks, max_applied_scn); } } else { - if (!callbacks.is_empty()) { - TRANS_LOG(INFO, "No memtable callbacks because of trans_end", K(end_code_), KPC(ctx_)); + if (callbacks.count() > 0) { + TRANS_LOG(INFO, "skip do callbacks because of trans_end", K(end_code_), KPC(ctx_)); + int fail_cnt = 0; + ARRAY_FOREACH(callbacks, i) { + fail_cnt += callbacks.at(i).cnt_; + } + log_gen_.inc_sync_log_fail_cnt(fail_cnt); } } return; } -uint64_t ObMemtableCtx::calc_checksum_all() +int ObMemtableCtx::calc_checksum_all(ObIArray &checksum) { ObByteLockGuard guard(lock_); - if (0 == trans_mgr_.get_checksum()) { - trans_mgr_.calc_checksum_all(); - } - return trans_mgr_.get_checksum(); + return trans_mgr_.calc_checksum_all(checksum); } void ObMemtableCtx::print_callbacks() { trans_mgr_.print_callbacks(); } +int ObMemtableCtx::get_callback_list_stat(ObIArray &stats) +{ + return trans_mgr_.get_callback_list_stat(stats); +} //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -822,14 +789,9 @@ uint64_t ObMemtableCtx::get_tenant_id() const return tenant_id; } -void ObMemtableCtx::update_max_submitted_seq_no(const transaction::ObTxSEQ seq_no) -{ - if (NULL != ATOMIC_LOAD(&ctx_)) { - static_cast(ctx_)->update_max_submitted_seq_no(seq_no); - } -} - -int ObMemtableCtx::rollback(const transaction::ObTxSEQ to_seq_no, const transaction::ObTxSEQ from_seq_no) +int ObMemtableCtx::rollback(const transaction::ObTxSEQ to_seq_no, + const transaction::ObTxSEQ from_seq_no, + const share::SCN replay_scn) { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("remove callbacks for rollback to", 10 * 1000); @@ -843,10 +805,10 @@ int ObMemtableCtx::rollback(const transaction::ObTxSEQ to_seq_no, const transact TRANS_LOG(WARN, "ctx is NULL", K(ret)); } else if (OB_FAIL(reuse_log_generator_())) { TRANS_LOG(ERROR, "fail to reset log generator", K(ret)); - } else if (OB_FAIL(trans_mgr_.rollback_to(to_seq_no, from_seq_no))) { + } else if (OB_FAIL(trans_mgr_.rollback_to(to_seq_no, from_seq_no, replay_scn))) { TRANS_LOG(WARN, "rollback to failed", K(ret), K(*this)); // rollback the table lock that with no tablelock callback - } else if (OB_FAIL(rollback_table_lock_(to_seq_no))) { + } else if (OB_FAIL(rollback_table_lock_(to_seq_no, from_seq_no))) { TRANS_LOG(WARN, "rollback table lock failed", K(ret), K(*this), K(to_seq_no)); } else { TRANS_LOG(INFO, "memtable handle rollback to successfuly", K(from_seq_no), K(to_seq_no), K(*this)); @@ -854,33 +816,27 @@ int ObMemtableCtx::rollback(const transaction::ObTxSEQ to_seq_no, const transact return ret; } -bool ObMemtableCtx::is_all_redo_submitted() -{ - ObByteLockGuard guard(lock_); - return trans_mgr_.is_all_redo_submitted(); -} - -int ObMemtableCtx::remove_callbacks_for_fast_commit() +int ObMemtableCtx::remove_callbacks_for_fast_commit(const int16_t callback_list_idx, const share::SCN stop_scn) { int ret = OB_SUCCESS; - bool meet_generate_cursor = false; common::ObTimeGuard timeguard("remove callbacks for fast commit", 10 * 1000); ObByteLockGuard guard(lock_); - - if (OB_FAIL(trans_mgr_.remove_callbacks_for_fast_commit( - log_gen_.get_generate_cursor(), - meet_generate_cursor))) { - TRANS_LOG(WARN, "fail to remove callback for uncommitted txn", K(ret), KPC(this)); - } else if (meet_generate_cursor && OB_FAIL(reuse_log_generator_())) { - TRANS_LOG(ERROR, "fail to reset log generator", K(ret), KPC(this)); + if (OB_FAIL(trans_mgr_.remove_callbacks_for_fast_commit(callback_list_idx, stop_scn))) { + TRANS_LOG(WARN, "fail to remove callback for fast commit", K(ret), KPC(this)); + } + return ret; +} +int ObMemtableCtx::remove_callbacks_for_fast_commit(const ObCallbackScopeArray &cb_scope_array) +{ + int ret = OB_SUCCESS; + common::ObTimeGuard timeguard("remove callbacks for fast commit", 10 * 1000); + if (OB_FAIL(trans_mgr_.remove_callbacks_for_fast_commit(cb_scope_array))) { + TRANS_LOG(WARN, "fail to remove callback for fast commit", K(ret)); } - return ret; } -int ObMemtableCtx::remove_callback_for_uncommited_txn( - const memtable::ObMemtableSet *memtable_set, - const share::SCN max_applied_scn) +int ObMemtableCtx::remove_callback_for_uncommited_txn(const memtable::ObMemtableSet *memtable_set) { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("remove callbacks for uncommitted txn", 10 * 1000); @@ -891,8 +847,7 @@ int ObMemtableCtx::remove_callback_for_uncommited_txn( TRANS_LOG(WARN, "memtable is NULL", K(memtable_set)); } else if (OB_FAIL(reuse_log_generator_())) { TRANS_LOG(ERROR, "fail to reset log generator", K(ret)); - } else if (OB_FAIL(trans_mgr_.remove_callback_for_uncommited_txn(memtable_set, - max_applied_scn))) { + } else if (OB_FAIL(trans_mgr_.remove_callback_for_uncommited_txn(memtable_set))) { TRANS_LOG(WARN, "fail to remove callback for uncommitted txn", K(ret), K(memtable_set)); } @@ -938,8 +893,8 @@ int ObMemtableCtx::reuse_log_generator_() } int ObMemtableCtx::calc_checksum_before_scn(const SCN scn, - uint64_t &checksum, - SCN &checksum_scn) + ObIArray &checksum, + ObIArray &checksum_scn) { int ret = OB_SUCCESS; ObByteLockGuard guard(lock_); @@ -951,22 +906,22 @@ int ObMemtableCtx::calc_checksum_before_scn(const SCN scn, return ret; } -void ObMemtableCtx::update_checksum(const uint64_t checksum, - const SCN checksum_scn) +int ObMemtableCtx::update_checksum(const ObIArray &checksum, + const ObIArray &checksum_scn) { ObByteLockGuard guard(lock_); - trans_mgr_.update_checksum(checksum, checksum_scn); + return trans_mgr_.update_checksum(checksum, checksum_scn); } -bool ObMemtableCtx::pending_log_size_too_large() +bool ObMemtableCtx::pending_log_size_too_large(const ObTxSEQ &write_seq_no) { bool ret = true; if (0 == GCONF._private_buffer_size) { ret = false; } else { - ret = trans_mgr_.get_pending_log_size() > GCONF._private_buffer_size; + ret = trans_mgr_.pending_log_size_too_large(write_seq_no, GCONF._private_buffer_size); } return ret; @@ -1002,7 +957,8 @@ int ObMemtableCtx::get_table_lock_for_transfer(ObTableLockInfo &table_lock_info, return ret; } -int ObMemtableCtx::recover_from_table_lock_durable_info(const ObTableLockInfo &table_lock_info) +int ObMemtableCtx::recover_from_table_lock_durable_info(const ObTableLockInfo &table_lock_info, + const bool transfer_merge) { int ret = OB_SUCCESS; const int64_t op_cnt = table_lock_info.table_lock_ops_.count(); @@ -1019,15 +975,15 @@ int ObMemtableCtx::recover_from_table_lock_durable_info(const ObTableLockInfo &t // the tx ctx table may be copied from other ls replica we need fix the lockop's create timestamp. } else if (FALSE_IT(lock_op.create_timestamp_ = OB_MIN(curr_timestamp, lock_op.create_timestamp_))) { - } else if (OB_FAIL(lock_mem_ctx_.add_lock_record(lock_op, lock_op_node, true))) { + } else if (OB_FAIL(lock_mem_ctx_.add_lock_record(lock_op, lock_op_node))) { TRANS_LOG(ERROR, "add_lock_record failed", K(ret), K(lock_op)); } else if (OB_FAIL(lock_mem_ctx_.get_lock_memtable(lock_memtable))) { TRANS_LOG(ERROR, "get_lock_memtable failed", K(ret)); } else if (OB_NOT_NULL(lock_memtable) && OB_FAIL(lock_memtable->recover_obj_lock(lock_op))) { TRANS_LOG(ERROR, "recover_obj_lock failed", K(ret), K(*lock_memtable)); - } else { - lock_mem_ctx_.set_log_synced(lock_op_node, table_lock_info.max_durable_scn_); + } else if (!transfer_merge) { + lock_mem_ctx_.sync_log_succ(table_lock_info.max_durable_scn_); } } @@ -1115,17 +1071,14 @@ int ObMemtableCtx::iterate_tx_obj_lock_op(ObLockOpIterator &iter) const int ObMemtableCtx::add_lock_record(const tablelock::ObTableLockOp &lock_op) { int ret = OB_SUCCESS; - const bool is_replay = false; ObMemCtxLockOpLinkNode *lock_op_node = nullptr; ObLockMemtable *memtable = nullptr; if (OB_UNLIKELY(!lock_op.is_valid())) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", K(ret), K(lock_op)); - } else if (OB_FAIL(lock_mem_ctx_.add_lock_record(lock_op, - lock_op_node, - is_replay))) { + } else if (OB_FAIL(lock_mem_ctx_.add_lock_record(lock_op, lock_op_node))) { TRANS_LOG(WARN, "create lock record at memtable failed. ", K(ret), K(lock_op), K(*this)); - } else if (OB_FAIL(register_multi_source_data_if_need_(lock_op, is_replay))) { + } else if (OB_FAIL(register_multi_source_data_if_need_(lock_op))) { TRANS_LOG(WARN, "register to multi source data failed", K(ret), K(lock_op)); } else if (OB_UNLIKELY(!lock_op.need_register_callback())) { // do nothing @@ -1147,15 +1100,12 @@ int ObMemtableCtx::replay_add_lock_record( const SCN &scn) { int ret = OB_SUCCESS; - const bool is_replay = true; ObMemCtxLockOpLinkNode *lock_op_node = nullptr; ObLockMemtable *memtable = nullptr; if (OB_UNLIKELY(!lock_op.is_valid())) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", K(ret), K(lock_op)); - } else if (OB_FAIL(lock_mem_ctx_.add_lock_record(lock_op, - lock_op_node, - is_replay))) { + } else if (OB_FAIL(lock_mem_ctx_.add_lock_record(lock_op, lock_op_node))) { TRANS_LOG(WARN, "create lock record at memtable failed. ", K(ret), K(lock_op), K(*this)); } else if (OB_UNLIKELY(!lock_op.need_register_callback())) { @@ -1170,7 +1120,7 @@ int ObMemtableCtx::replay_add_lock_record( } else { // make sure the replayed tablelock will be minor merged. // and update the max durable log ts. - lock_mem_ctx_.set_log_synced(lock_op_node, scn); + lock_mem_ctx_.sync_log_succ(scn); } if (OB_FAIL(ret) && lock_op_node != NULL) { lock_mem_ctx_.remove_lock_record(lock_op_node); @@ -1183,11 +1133,6 @@ void ObMemtableCtx::remove_lock_record(ObMemCtxLockOpLinkNode *lock_op) lock_mem_ctx_.remove_lock_record(lock_op); } -void ObMemtableCtx::set_log_synced(ObMemCtxLockOpLinkNode *lock_op, const SCN &scn) -{ - lock_mem_ctx_.set_log_synced(lock_op, scn); -} - int ObMemtableCtx::clear_table_lock_(const bool is_commit, const SCN &commit_version, const SCN &commit_scn) @@ -1204,12 +1149,13 @@ int ObMemtableCtx::clear_table_lock_(const bool is_commit, return ret; } -int ObMemtableCtx::rollback_table_lock_(transaction::ObTxSEQ seq_no) +int ObMemtableCtx::rollback_table_lock_(transaction::ObTxSEQ to_seq_no, + transaction::ObTxSEQ from_seq_no) { int ret = OB_SUCCESS; if (is_read_only_) { // read only trx no need deal with table lock. - } else if (OB_FAIL(lock_mem_ctx_.rollback_table_lock(seq_no))) { + } else if (OB_FAIL(lock_mem_ctx_.rollback_table_lock(to_seq_no, from_seq_no))) { TRANS_LOG(WARN, "clear table lock failed", KP(this)); } @@ -1217,16 +1163,12 @@ int ObMemtableCtx::rollback_table_lock_(transaction::ObTxSEQ seq_no) } int ObMemtableCtx::register_multi_source_data_if_need_( - const tablelock::ObTableLockOp &lock_op, - const bool is_replay) + const tablelock::ObTableLockOp &lock_op) { int ret = OB_SUCCESS; ObRegisterMdsFlag mds_flag; mds_flag.reset(); - if (is_replay) { - // replay does not need register multi source data, it is dealt - // by multi source data itself. - } else if (!lock_op.need_multi_source_data()) { + if (!lock_op.need_multi_source_data()) { // do nothing } else if (OB_ISNULL(ATOMIC_LOAD(&ctx_))) { // if a trans need lock table, it must not a readonly trans. @@ -1297,16 +1239,6 @@ void ObMemtableCtx::dec_unsubmitted_cnt() ATOMIC_DEC(&unsubmitted_cnt_); } -void ObMemtableCtx::inc_unsynced_cnt() -{ - ATOMIC_INC(&unsynced_cnt_); -} - -void ObMemtableCtx::dec_unsynced_cnt() -{ - ATOMIC_DEC(&unsynced_cnt_); -} - int ObMemtableCtx::check_tx_mem_size_overflow(bool &is_overflow) { int ret = OB_SUCCESS; @@ -1326,5 +1258,43 @@ void ObMemtableCtx::print_first_mvcc_callback() log_gen_.print_first_mvcc_callback(); } +void ObMemtableCtx::convert_checksum_for_commit_log(const ObIArray &arr, + uint64_t &checksum, + ObIArray &sig) +{ + // NOTE: + // should skip empty callback-list, because it will not be replayed out on follower + if (arr.count() == 0) { + checksum = 0; + sig.reset(); + } else if (arr.count() == 1) { + checksum = arr.at(0); + sig.reset(); + } else { + int valid_cnt = 0; + for (int i = 0; i < arr.count(); i++) { + if (arr.at(i) && arr.at(i) != 1) { // skip empty list whose checksum is '1' + ++valid_cnt; + if (valid_cnt == 1) { + checksum = arr.at(i); + } else { + checksum = ob_crc64(checksum, (void*)&arr.at(i), sizeof(uint64_t)); + } + } + sig.push_back((uint8_t)(arr.at(i) & 0xFF)); + } + if (valid_cnt == 0) { + checksum = arr.at(0); + sig.reset(); + } else if (valid_cnt == 1) { + sig.reset(); + } + } +} + +void ObMemtableCtx::check_all_redo_flushed() +{ + trans_mgr_.check_all_redo_flushed(); +} } } diff --git a/src/storage/memtable/ob_memtable_context.h b/src/storage/memtable/ob_memtable_context.h index 8d9af1b238..a1c101c951 100644 --- a/src/storage/memtable/ob_memtable_context.h +++ b/src/storage/memtable/ob_memtable_context.h @@ -44,7 +44,7 @@ struct ObTableLockInfo; namespace memtable { - +class ObTxCallbackListStat; struct RetryInfo { RetryInfo() : retry_cnt_(0), last_retry_ts_(0) {} @@ -349,8 +349,6 @@ public: return ATOMIC_LOAD(&end_code_); } virtual int write_lock_yield(); - - virtual void update_max_submitted_seq_no(const transaction::ObTxSEQ seq_no) override; public: virtual void set_read_only(); virtual void inc_ref(); @@ -360,11 +358,15 @@ public: virtual int write_auth(const bool exclusive); virtual int write_done(); virtual int trans_begin(); - virtual int replay_begin(const share::SCN scn); + virtual int replay_begin(const bool parallel_replay, const share::SCN scn); virtual int replay_end(const bool is_replay_succ, + const int16_t callback_list_idx, const share::SCN scn); - int rollback_redo_callbacks(const share::SCN scn); - virtual uint64_t calc_checksum_all(); + int rollback_redo_callbacks(const int16_t callback_list_idx, const share::SCN scn); + virtual int calc_checksum_all(ObIArray &checksum); + static void convert_checksum_for_commit_log(const ObIArray &arr, + uint64_t &checksum, + ObIArray &sig); virtual void print_callbacks(); virtual int trans_end(const bool commit, const share::SCN trans_version, @@ -383,23 +385,19 @@ public: virtual int replay_to_commit(const bool is_resume); //method called when leader revoke virtual int commit_to_replay(); - virtual int fill_redo_log(char *buf, - const int64_t buf_len, - int64_t &buf_pos, - ObRedoLogSubmitHelper &helper, - const bool log_for_lock_node = true); + virtual int fill_redo_log(ObTxFillRedoCtx &ctx); + void check_all_redo_flushed(); + int get_log_guard(const transaction::ObTxSEQ &write_seq, + ObCallbackListLogGuard &log_guard, + int &cb_list_idx); int calc_checksum_before_scn(const share::SCN scn, - uint64_t &checksum, - share::SCN &checksum_scn); - void update_checksum(const uint64_t checksum, - const share::SCN checksum_scn); + ObIArray &checksum, + ObIArray &checksum_scn); + int update_checksum(const ObIArray &checksum, + const ObIArray &checksum_scn); int log_submitted(const ObRedoLogSubmitHelper &helper); - // the function apply the side effect of dirty txn and return whether - // remaining pending callbacks. - // NB: the fact whether there remains pending callbacks currently is only used - // for continuing logging when minor freeze - int sync_log_succ(const share::SCN scn, const ObCallbackScope &callbacks); - void sync_log_fail(const ObCallbackScope &callbacks); + int sync_log_succ(const share::SCN scn, const ObCallbackScopeArray &callbacks); + void sync_log_fail(const ObCallbackScopeArray &callbacks, const share::SCN &max_applied_scn); bool is_slow_query() const; virtual void set_trans_ctx(transaction::ObPartTransCtx *ctx); virtual transaction::ObPartTransCtx *get_trans_ctx() const { return ctx_; } @@ -411,19 +409,24 @@ public: uint64_t get_tenant_id() const; inline bool has_row_updated() const { return has_row_updated_; } inline void set_row_updated() { has_row_updated_ = true; } - int remove_callbacks_for_fast_commit(); - int remove_callback_for_uncommited_txn( - const memtable::ObMemtableSet *memtable_set, - const share::SCN max_applied_scn); - int rollback(const transaction::ObTxSEQ seq_no, const transaction::ObTxSEQ from_seq_no); - bool is_all_redo_submitted(); + int remove_callbacks_for_fast_commit(const ObCallbackScopeArray &callbacks); + int remove_callbacks_for_fast_commit(const int16_t callback_list_idx, const share::SCN stop_scn); + int remove_callback_for_uncommited_txn(const memtable::ObMemtableSet *memtable_set); + int rollback(const transaction::ObTxSEQ seq_no, const transaction::ObTxSEQ from_seq_no, + const share::SCN replay_scn); + void set_parallel_logging(const share::SCN serial_final_scn) { + trans_mgr_.set_parallel_logging(serial_final_scn); + } + void set_skip_checksum_calc() { + trans_mgr_.set_skip_checksum_calc(); + } bool is_for_replay() const { return trans_mgr_.is_for_replay(); } int64_t get_trans_mem_total_size() const { return trans_mem_total_size_; } void add_lock_for_read_elapse(const int64_t elapse) { lock_for_read_elapse_ += elapse; } int64_t get_lock_for_read_elapse() const { return lock_for_read_elapse_; } int64_t get_pending_log_size() { return trans_mgr_.get_pending_log_size(); } int64_t get_flushed_log_size() { return trans_mgr_.get_flushed_log_size(); } - bool pending_log_size_too_large(); + bool pending_log_size_too_large(const transaction::ObTxSEQ &write_seq_no); void merge_multi_callback_lists_for_changing_leader(); void merge_multi_callback_lists_for_immediate_logging(); void reset_pdml_stat(); @@ -445,11 +448,6 @@ public: // statics maintainness for txn logging virtual void inc_unsubmitted_cnt() override; virtual void dec_unsubmitted_cnt() override; - virtual void inc_unsynced_cnt() override; - virtual void dec_unsynced_cnt() override; - int64_t get_checksum() const { return trans_mgr_.get_checksum(); } - int64_t get_tmp_checksum() const { return trans_mgr_.get_tmp_checksum(); } - share::SCN get_checksum_scn() const { return trans_mgr_.get_checksum_scn(); } public: // tx_status enum ObTxStatus { @@ -464,6 +462,7 @@ public: public: // table lock. int enable_lock_table(storage::ObTableHandleV2 &handle); + transaction::tablelock::ObLockMemCtx &get_lock_mem_ctx() { return lock_mem_ctx_; } int check_lock_exist(const ObLockID &lock_id, const ObTableLockOwnerID &owner_id, const ObTableLockMode mode, @@ -482,12 +481,12 @@ public: int replay_add_lock_record(const transaction::tablelock::ObTableLockOp &lock_op, const share::SCN &scn); void remove_lock_record(ObMemCtxLockOpLinkNode *lock_op); - void set_log_synced(ObMemCtxLockOpLinkNode *lock_op, const share::SCN &scn); // replay lock to lock map and trans part ctx. // used by the replay process of multi data source. int replay_lock(const transaction::tablelock::ObTableLockOp &lock_op, const share::SCN &scn); - int recover_from_table_lock_durable_info(const ObTableLockInfo &table_lock_info); + int recover_from_table_lock_durable_info(const ObTableLockInfo &table_lock_info, + const bool transfer_merge = false); int get_table_lock_store_info(ObTableLockInfo &table_lock_info); int get_table_lock_for_transfer(ObTableLockInfo &table_lock_info, const ObIArray &tablet_list); // for deadlock detect. @@ -501,7 +500,7 @@ public: trans_mgr_.get_callback_remove_for_remove_memtable_count() > 0; } void print_first_mvcc_callback(); - + int get_callback_list_stat(ObIArray &stats); private: int do_trans_end( const bool commit, @@ -511,10 +510,10 @@ private: int clear_table_lock_(const bool is_commit, const share::SCN &commit_version, const share::SCN &commit_scn); - int rollback_table_lock_(transaction::ObTxSEQ seq_no); + int rollback_table_lock_(const transaction::ObTxSEQ to_seq_no, + const transaction::ObTxSEQ from_seq_no); int register_multi_source_data_if_need_( - const transaction::tablelock::ObTableLockOp &lock_op, - const bool is_replay); + const transaction::tablelock::ObTableLockOp &lock_op); static int64_t get_us() { return ::oceanbase::common::ObTimeUtility::current_time(); } int reset_log_generator_(); int reuse_log_generator_(); @@ -548,7 +547,6 @@ private: int64_t lock_for_read_elapse_; int64_t trans_mem_total_size_; // statistics for txn logging - int64_t unsynced_cnt_; int64_t unsubmitted_cnt_; int64_t callback_mem_used_; int64_t callback_alloc_count_; diff --git a/src/storage/memtable/ob_memtable_interface.h b/src/storage/memtable/ob_memtable_interface.h index 107a0e3a2a..909e9a8fe1 100644 --- a/src/storage/memtable/ob_memtable_interface.h +++ b/src/storage/memtable/ob_memtable_interface.h @@ -40,7 +40,7 @@ class ObTransID; } namespace memtable { - +class ObTxFillRedoCtx; class ObRedoLogSubmitHelper; class ObMemtableCtxCbAllocator; @@ -83,11 +83,7 @@ public: VIRTUAL_TO_STRING_KV("", ""); public: // return OB_AGAIN/OB_SUCCESS - virtual int fill_redo_log(char *buf, - const int64_t buf_len, - int64_t &buf_pos, - ObRedoLogSubmitHelper &helper, - const bool log_for_lock_node = true) = 0; + virtual int fill_redo_log(ObTxFillRedoCtx &ctx) = 0; common::ActiveResource resource_link_; }; diff --git a/src/storage/memtable/ob_redo_log_generator.cpp b/src/storage/memtable/ob_redo_log_generator.cpp index 7228d48ef4..6e49d86f81 100644 --- a/src/storage/memtable/ob_redo_log_generator.cpp +++ b/src/storage/memtable/ob_redo_log_generator.cpp @@ -16,6 +16,7 @@ #include "ob_memtable.h" #include "ob_memtable_data.h" #include "ob_memtable_context.h" +#include "mvcc/ob_tx_callback_functor.h" #include "storage/tx/ob_trans_part_ctx.h" #include "storage/tablelock/ob_table_lock_callback.h" @@ -32,7 +33,6 @@ void ObRedoLogGenerator::reset() redo_filled_cnt_ = 0; redo_sync_succ_cnt_ = 0; redo_sync_fail_cnt_ = 0; - generate_cursor_.reset(); callback_mgr_ = nullptr; mem_ctx_ = NULL; last_logging_blocked_time_ = 0; @@ -44,11 +44,10 @@ void ObRedoLogGenerator::reset() void ObRedoLogGenerator::reuse() { - generate_cursor_.reset(); - generate_cursor_ = callback_mgr_->begin(); + //TODO: remove this } -int ObRedoLogGenerator::set(ObTransCallbackMgr *mgr, ObIMemtableCtx *mem_ctx) +int ObRedoLogGenerator::set(ObTransCallbackMgr *mgr, ObMemtableCtx *mem_ctx) { if (IS_INIT) { // already set, reset first @@ -56,7 +55,6 @@ int ObRedoLogGenerator::set(ObTransCallbackMgr *mgr, ObIMemtableCtx *mem_ctx) } int ret = OB_SUCCESS; - generate_cursor_ = mgr->begin(); callback_mgr_ = mgr; mem_ctx_ = mem_ctx; last_logging_blocked_time_ = 0; @@ -65,217 +63,257 @@ int ObRedoLogGenerator::set(ObTransCallbackMgr *mgr, ObIMemtableCtx *mem_ctx) return ret; } -int ObRedoLogGenerator::fill_redo_log(char *buf, - const int64_t buf_len, - int64_t &buf_pos, - ObRedoLogSubmitHelper &helper, - const bool log_for_lock_node) +// +// this functor handle _one_ callback +// +// return value: +// - OB_SUCCESS: success, all callbacks were filled +// - OB_BUF_NOT_ENOUGH: buffer can not hold this callback +// - OB_BLOCK_FROZEN: the callback's memtable logging is blocked +// on waiting the previous frozen siblings logged +// - OB_ITER_END: reach end of *ctx.epoch_to_* +// - OB_XXX: other error +class ObFillRedoLogFunctor final : public ObITxFillRedoFunctor +{ +public: + ObFillRedoLogFunctor(ObMemtableCtx *mem_ctx, + transaction::ObTxEncryptMeta *clog_encrypt_meta, + ObTxFillRedoCtx &ctx, + ObMutatorWriter &mmw, + transaction::ObCLogEncryptInfo &encrypt_info) : + mem_ctx_(mem_ctx), + clog_encrypt_meta_(clog_encrypt_meta), + ctx_(ctx), + mmw_(mmw), + encrypt_info_(encrypt_info) + {} + int operator()(ObITransCallback *iter) + { + TRANS_LOG(DEBUG, "fill_redo_for_callback", KPC(iter)); + int ret = OB_SUCCESS; + if (iter->get_epoch() > ctx_.epoch_to_) { + ret = OB_ITER_END; + ctx_.next_epoch_ = iter->get_epoch(); + } else if (iter->get_epoch() < ctx_.epoch_from_) { + ret = OB_ERR_UNEXPECTED; + ob_abort(); + } else if (FALSE_IT(ctx_.cur_epoch_ = iter->get_epoch())) { + } else if (!iter->need_submit_log()) { + // this should not happend + // because log_cursor is _strictly_ point to the right next to logging position + ob_abort(); + } else if (iter->is_logging_blocked()) { + ret = OB_BLOCK_FROZEN; + ctx_.last_log_blocked_memtable_ = static_cast(iter->get_memtable()); + // TODO:(yunxing.cyx) find a way let redo submitter can quickly check BLOCK_FROZEN + // comment out the following piece of code, it not works under multiple CallbackList: + /************************************************************************************ + if (data_node_count == 0) { + // To prevent unnecessary submit_log actions for freeze + // Becasue the first callback is linked to a logging_blocked memtable + transaction::ObPartTransCtx *part_ctx = static_cast(mem_ctx_->get_trans_ctx()); + part_ctx->set_block_frozen_memtable(static_cast(iter->get_memtable())); + int64_t current_time = ObTimeUtility::current_time(); + if (last_logging_blocked_time_ == 0) { + last_logging_blocked_time_ = current_time; + } else if (current_time - last_logging_blocked_time_ > 5 * 1_min) { + TRANS_LOG(WARN, "logging block cost too much time", KPC(part_ctx), KPC(iter)); + if (REACH_TENANT_TIME_INTERVAL(1_min)) { + bug_detect_for_logging_blocked_(); + } + } + } + ***********************************************************************************/ + } else { + bool fake_fill = false; + if (MutatorType::MUTATOR_ROW == iter->get_mutator_type()) { + ret = fill_row_redo_(iter, fake_fill); + } else if (MutatorType::MUTATOR_TABLE_LOCK == iter->get_mutator_type()) { + ret = fill_table_lock_redo_(iter, fake_fill); + } else { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "mutator row type not expected.", K(ret)); + } + + if (OB_SUCC(ret)) { + ObCallbackScope &callback_scope = *ctx_.callback_scope_; + if (nullptr == *callback_scope.start_) { + callback_scope.start_ = iter; + } + callback_scope.end_ = iter; + ++callback_scope.cnt_; + if (!fake_fill) { + ctx_.fill_count_++; + } + data_size_ += iter->get_data_size(); + max_seq_no_ = MAX(max_seq_no_, iter->get_seq_no()); + } + } + return ret; + } +private: + int fill_row_redo_(ObITransCallback *callback, bool &fake_fill) + { + int ret = OB_SUCCESS; + RedoDataNode redo; + ObMvccRowCallback *riter = (ObMvccRowCallback *)callback; + + if (blocksstable::ObDmlFlag::DF_LOCK == riter->get_dml_flag()) { + if (ctx_.skip_lock_node_) { + riter->set_not_calc_checksum(true); + fake_fill = true; + } else { + // need to calc checksum + riter->set_not_calc_checksum(false); + } + } + + if (fake_fill) { + } else if (OB_FAIL(riter->get_redo(redo)) && OB_ENTRY_NOT_EXIST != ret) { + TRANS_LOG(ERROR, "get_redo", K(ret)); + } else if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + ObMemtable *memtable = static_cast(riter->get_memtable()); + if (OB_ISNULL(memtable)) { + TRANS_LOG(ERROR, "memtable is null", K(riter)); + ret = OB_ERR_UNEXPECTED; +#ifdef OB_BUILD_TDE_SECURITY + } else if (OB_FAIL(memtable->get_encrypt_meta(clog_encrypt_meta_))) { + TRANS_LOG(ERROR, "get encrypt meta failed", K(memtable), K(ret)); +#endif + } else if (OB_FAIL(mmw_.append_row_kv(mem_ctx_->get_max_table_version(), + redo, + clog_encrypt_meta_, + encrypt_info_, + false))) { + if (OB_BUF_NOT_ENOUGH != ret) { + TRANS_LOG(WARN, "mutator writer append_kv fail", "ret", ret); + } + } + } + return ret; + } + int fill_table_lock_redo_(ObITransCallback *callback, bool &fake_fill) + { + int ret = OB_SUCCESS; + TableLockRedoDataNode redo; + ObOBJLockCallback *titer = (ObOBJLockCallback *)callback; + if (ctx_.skip_lock_node_ && !titer->must_log()) { + fake_fill = true; + } else if (OB_FAIL(titer->get_redo(redo))) { + TRANS_LOG(ERROR, "get_redo failed.", K(ret)); + } else if (OB_FAIL(mmw_.append_table_lock_kv(mem_ctx_->get_max_table_version(), redo))) { + if (OB_BUF_NOT_ENOUGH != ret) { + TRANS_LOG(WARN, "fill table lock redo fail", K(ret)); + } + } + TRANS_LOG(DEBUG, "fill table lock redo.", K(ret), K(*titer), K(redo.lock_id_), K(redo.lock_mode_)); + return ret; + } +private: + ObMemtableCtx *mem_ctx_; + transaction::ObTxEncryptMeta *clog_encrypt_meta_; + ObTxFillRedoCtx &ctx_; + ObMutatorWriter &mmw_; + transaction::ObCLogEncryptInfo &encrypt_info_; +}; + +// +// fill redo log into log block +// +// This handle both serial logging and parallel logging +// for serial logging: +// callbacks from multi callback-list filled together into one log block +// for parallel logging: +// each callback-list's logs are filled into seperate log block +// +// In parallel logging mode, there are two type of fill_redo scheme: +// 1. fill from all callback-list: +// for freeze, switch leader, commit +// 2. fill from single callback-list: +// writer thread flush pending logs from its callback-list after write +// +// return value: +// - OB_SUCCESS: all callbacks are filled +// - OB_BUF_NOT_ENOUGH: buffer is full or can not hold mutator row +// - OB_BLOCK_FROZEN: the callback's memtable is blocked, can not be fill +// - OB_ITER_END: has small write_epoch whose log is not flushed +int ObRedoLogGenerator::fill_redo_log(ObTxFillRedoCtx &ctx) { int ret = OB_SUCCESS; - - if (OB_ISNULL(buf) || buf_len < 0 || buf_pos < 0 || buf_pos > buf_len) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid_argument", KP(buf), K(buf_len), K(buf_pos)); - } else if (IS_NOT_INIT) { + if (IS_NOT_INIT) { ret = OB_NOT_INIT; + } else if (OB_ISNULL(ctx.buf_) || ctx.buf_len_ < 0) { + ret = OB_INVALID_ARGUMENT; + TRANS_LOG(WARN, "invalid_argument", KP(ctx.buf_), K(ctx.buf_len_)); } else { - helper.reset(); + // prepare the global varibles ObMutatorWriter mmw; - mmw.set_buffer(buf, buf_len - buf_pos); - RedoDataNode redo; - TableLockRedoDataNode table_lock_redo; - // record the number of serialized trans node in the filling process - int64_t data_node_count = 0; - transaction::ObTxSEQ max_seq_no; - // TODO by fengshuo.fs : fix this usage - ObTransCallbackMgr::RDLockGuard guard(callback_mgr_->get_rwlock()); - ObCallbackScope callbacks; - int64_t data_size = 0; - ObITransCallbackIterator cursor; - // for encrypt + mmw.set_buffer(ctx.buf_, ctx.buf_len_ - ctx.buf_pos_); + // used to encrypt each mutator row transaction::ObCLogEncryptInfo encrypt_info; encrypt_info.init(); + ctx.helper_->reset(); + ObFillRedoLogFunctor functor(mem_ctx_, clog_encrypt_meta_, ctx, mmw, encrypt_info); + ret = callback_mgr_->fill_log(ctx, functor); - for (cursor = generate_cursor_ + 1; OB_SUCC(ret) && callback_mgr_->end() != cursor; ++cursor) { - ObITransCallback *iter = (ObITransCallback *)*cursor; - - if (!iter->need_fill_redo() || !iter->need_submit_log()) { - } else if (iter->is_logging_blocked()) { - ret = (data_node_count == 0) ? OB_BLOCK_FROZEN : OB_EAGAIN; - if (OB_BLOCK_FROZEN == ret) { - // To prevent unnecessary submit_log actions for freeze - // Becasue the first callback is linked to a logging_blocked memtable - transaction::ObPartTransCtx *part_ctx = static_cast(mem_ctx_->get_trans_ctx()); - part_ctx->set_block_frozen_memtable(static_cast(iter->get_memtable())); - - int64_t current_time = ObTimeUtility::current_time(); - if (last_logging_blocked_time_ == 0) { - last_logging_blocked_time_ = current_time; - } else if (current_time - last_logging_blocked_time_ > 5 * 1_min) { - TRANS_LOG(WARN, "logging block cost too much time", KPC(part_ctx), KPC(iter)); - if (REACH_TENANT_TIME_INTERVAL(1_min)) { - bug_detect_for_logging_blocked_(); - } - } - } - } else { - last_logging_blocked_time_ = 0; - - bool fake_fill = false; - if (MutatorType::MUTATOR_ROW == iter->get_mutator_type()) { - ret = fill_row_redo(cursor, mmw, redo, log_for_lock_node, fake_fill, encrypt_info); - } else if (MutatorType::MUTATOR_TABLE_LOCK == iter->get_mutator_type()) { - ret = fill_table_lock_redo(cursor, mmw, table_lock_redo, log_for_lock_node, fake_fill); - } else { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "mutator row type not expected.", K(ret)); - } - - if (OB_BUF_NOT_ENOUGH == ret) { - // buf is not enough: if some rows have been serialized before, that means - // more redo data is demanding more buf, returns OB_EAGAIN; - // if the buf is not enough for the first trans node, that means a big row - // is comming, handle it according to the big row logic - if (0 != data_node_count) { - ret = OB_EAGAIN; - // deal with big row logic - } else { - ret = OB_ERR_TOO_BIG_ROWSIZE; - } - } - - if (OB_UNLIKELY(OB_ERR_TOO_BIG_ROWSIZE == ret)) { - callbacks.start_ = callbacks.end_ = cursor; - data_size += iter->get_data_size(); - max_seq_no = MAX(max_seq_no, iter->get_seq_no()); - } else if (OB_SUCC(ret)) { - if (nullptr == *callbacks.start_) { - callbacks.start_ = cursor; - } - callbacks.end_ = cursor; - - if (!fake_fill) { - data_node_count++; - } - data_size += iter->get_data_size(); - max_seq_no = MAX(max_seq_no, iter->get_seq_no()); - } - } - } - - if (OB_EAGAIN == ret || OB_SUCCESS == ret || OB_ERR_TOO_BIG_ROWSIZE == ret) { - int tmp_ret = OB_SUCCESS; - - helper.callbacks_ = callbacks; - helper.max_seq_no_ = max_seq_no; - helper.data_size_ = data_size; - - if (OB_LIKELY(OB_ERR_TOO_BIG_ROWSIZE != ret)) { - int64_t res_len = 0; - uint8_t row_flag = ObTransRowFlag::NORMAL_ROW; + // finally, serialize meta and finish the RedoLog + int save_ret = ret; + if (ctx.fill_count_ > 0) { + int64_t res_len = 0; + uint8_t row_flag = ObTransRowFlag::NORMAL_ROW; #ifdef OB_BUILD_TDE_SECURITY - if (encrypt_info.has_encrypt_meta()) { - row_flag |= ObTransRowFlag::ENCRYPT; - } + if (encrypt_info.has_encrypt_meta()) { + row_flag |= ObTransRowFlag::ENCRYPT; + } #endif - if (OB_SUCCESS != (tmp_ret = mmw.serialize(row_flag, res_len, encrypt_info))) { - if (OB_ENTRY_NOT_EXIST != tmp_ret) { - TRANS_LOG(ERROR, "mmw.serialize fail", K(ret), K(tmp_ret)); - ret = tmp_ret; - } else { - #ifndef NDEBUG - TRANS_LOG(INFO, "not row exist, ignore serialize", K(ret), K(tmp_ret), K(mmw.get_meta()), - K(data_node_count)); - #endif - if (OB_SUCCESS == ret) { - ret = tmp_ret; - } - } - } else { - buf_pos += res_len; - } + if (OB_FAIL(mmw.serialize(row_flag, res_len, encrypt_info))) { + TRANS_LOG(WARN, "mmw.serialize fail, can not submit this redo out", K(ret)); + // if serialize meta failed, this round of fill redo failed + // mark the fill_count_ to indicate this + ctx.fill_count_ = 0; + } else { + ctx.buf_pos_ += res_len; + ret = save_ret; } } } return ret; } -// sub unsubmitted cnt for the callback that has submitted log -int ObRedoLogGenerator::log_submitted(const ObCallbackScope &callbacks) +// log_submitted - callback after log submitted +// - mark callback's state to log submitted +// - mark callback's log scn +int ObRedoLogGenerator::log_submitted(const ObCallbackScopeArray &callbacks_arr, const share::SCN &scn) { int ret = OB_SUCCESS; - int tmp_ret = OB_SUCCESS; - + int submitted_cnt = 0; if (IS_NOT_INIT) { ret = OB_NOT_INIT; TRANS_LOG(ERROR, "not init", K(ret)); - } else if (!callbacks.is_empty()) { - ObITransCallbackIterator cursor = callbacks.start_; - do { - ObITransCallback *iter = (ObITransCallback *)*cursor; - if (iter->need_submit_log()) { - if (OB_TMP_FAIL(iter->log_submitted_cb())) { - if (OB_SUCC(ret)) { - ret = tmp_ret; - } - TRANS_LOG(ERROR, "fail to log_submitted cb", K(tmp_ret)); - } else { - redo_filled_cnt_ += 1; - } - // check dup table tx - if(check_dup_tablet_(iter)) - { - // mem_ctx_->get_trans_ctx()->set_dup_table_tx_(); - } - } else { - TRANS_LOG(ERROR, "log_submitted error", K(ret), K(iter), K(iter->need_submit_log())); - } - } while (cursor != callbacks.end_ && !FALSE_IT(cursor++)); - - generate_cursor_ = callbacks.end_; + } else if (OB_FAIL(callback_mgr_->log_submitted(callbacks_arr, scn, submitted_cnt))) { + TRANS_LOG(ERROR, "log submitted callback fail", K(ret)); } - + ATOMIC_AAF(&redo_filled_cnt_, submitted_cnt); return ret; } -int ObRedoLogGenerator::sync_log_succ(const SCN scn, const ObCallbackScope &callbacks) +int ObRedoLogGenerator::sync_log_succ(const ObCallbackScopeArray &callbacks_arr, + const share::SCN &scn) { - // no need to submit log - // since the number of log callback is enough now - // and will be allocated dynamically in the future int ret = OB_SUCCESS; - int tmp_ret = OB_SUCCESS; - - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - TRANS_LOG(ERROR, "not init", K(ret)); - } else if (!callbacks.is_empty()) { - ObTransCallbackMgr::RDLockGuard guard(callback_mgr_->get_rwlock()); - ObITransCallbackIterator cursor = callbacks.start_; - do { - ObITransCallback *iter = (ObITransCallback *)*cursor; - if (iter->need_fill_redo()) { - iter->set_scn(scn); - if (OB_TMP_FAIL(iter->log_sync_cb(scn))) { - if (OB_SUCC(ret)) { - ret = tmp_ret; - } - TRANS_LOG(WARN, "failed to set sync log info for callback ", K(tmp_ret), K(*iter)); - } else { - redo_sync_succ_cnt_ += 1; - } - } else { - TRANS_LOG(ERROR, "sync_log_succ error", K(ret), K(iter), K(iter->need_fill_redo()), K(scn)); - } - } while (cursor != callbacks.end_ && !FALSE_IT(cursor++)); + int64_t sync_cnt = 0; + if (OB_FAIL(callback_mgr_->log_sync_succ(callbacks_arr, scn, sync_cnt))) { + TRANS_LOG(ERROR, "sync succ fail", K(ret)); + } else { + redo_sync_succ_cnt_ += sync_cnt; } - // TRANS_LOG(INFO, "sync log succ for memtable callbacks", K(ret), K(redo_sync_succ_cnt_), "tx_id", - // this->mem_ctx_->get_trans_ctx()->get_trans_id()); - return ret; } -void ObRedoLogGenerator::sync_log_fail(const ObCallbackScope &callbacks) +void ObRedoLogGenerator::sync_log_fail(const ObCallbackScopeArray &callbacks, + const share::SCN &max_applied_scn) { int ret = OB_SUCCESS; int64_t removed_cnt = 0; @@ -283,10 +321,9 @@ void ObRedoLogGenerator::sync_log_fail(const ObCallbackScope &callbacks) if (IS_NOT_INIT) { ret = OB_NOT_INIT; TRANS_LOG(ERROR, "not init", K(ret)); - } else if (!callbacks.is_empty()) { - if (OB_FAIL(callback_mgr_->sync_log_fail(callbacks, removed_cnt))) { - TRANS_LOG(ERROR, "sync log failed", K(ret)); - } + } else if (OB_FAIL(callback_mgr_->log_sync_fail(callbacks, max_applied_scn, removed_cnt))) { + TRANS_LOG(ERROR, "sync log failed", K(ret)); + } else { redo_sync_fail_cnt_ += removed_cnt; } } @@ -297,119 +334,41 @@ void ObRedoLogGenerator::print_first_mvcc_callback() if (IS_NOT_INIT) { ret = OB_NOT_INIT; } else { - ObTransCallbackMgr::RDLockGuard guard(callback_mgr_->get_rwlock()); - ObITransCallbackIterator cursor = generate_cursor_ + 1; - if (callback_mgr_->end() != cursor) { - ObITransCallback *iter = (ObITransCallback *)*cursor; - LOG_DBA_WARN(OB_TRANS_LIVE_TOO_MUCH_TIME, - "msg", - "transaction live cost too much time without commit or abort", - KPC(iter)); - } + // TODO:(yunxing.cyx) PRINT ... } } -int ObRedoLogGenerator::fill_row_redo(ObITransCallbackIterator &cursor, - ObMutatorWriter &mmw, - RedoDataNode &redo, - const bool log_for_lock_node, - bool &fake_fill, - transaction::ObCLogEncryptInfo &encrypt_info) -{ - int ret = OB_SUCCESS; - - ObMvccRowCallback *riter = (ObMvccRowCallback *)*cursor; - - if (blocksstable::ObDmlFlag::DF_LOCK == riter->get_dml_flag()) { - if (!log_for_lock_node) { - riter->set_not_calc_checksum(true); - fake_fill = true; - } else { - // need to calc checksum - riter->set_not_calc_checksum(false); - } - } - - if (fake_fill) { - } else if (OB_FAIL(riter->get_redo(redo)) && OB_ENTRY_NOT_EXIST != ret) { - TRANS_LOG(ERROR, "get_redo", K(ret)); - } else if (OB_ENTRY_NOT_EXIST == ret) { - ret = OB_SUCCESS; - } else { - ObMemtable *memtable = static_cast(riter->get_memtable()); - if (OB_ISNULL(memtable)) { - TRANS_LOG(ERROR, "memtable is null", K(riter)); - ret = OB_ERR_UNEXPECTED; -#ifdef OB_BUILD_TDE_SECURITY - } else if (OB_FAIL(memtable->get_encrypt_meta(clog_encrypt_meta_))) { - TRANS_LOG(ERROR, "get encrypt meta failed", K(memtable), K(ret)); -#endif - } else if (OB_FAIL(mmw.append_row_kv(mem_ctx_->get_max_table_version(), - redo, - clog_encrypt_meta_, - encrypt_info, - false))) { - if (OB_BUF_NOT_ENOUGH != ret) { - TRANS_LOG(WARN, "mutator writer append_kv fail", "ret", ret); - } - } - } - - return ret; -} - -int ObRedoLogGenerator::fill_table_lock_redo(ObITransCallbackIterator &cursor, - ObMutatorWriter &mmw, - TableLockRedoDataNode &redo, - const bool log_for_lock_node, - bool &fake_fill) -{ - int ret = OB_SUCCESS; - - ObOBJLockCallback *titer = (ObOBJLockCallback *)*cursor; - if (!log_for_lock_node && !titer->must_log()) { - fake_fill = true; - } else if (OB_FAIL(titer->get_redo(redo))) { - TRANS_LOG(ERROR, "get_redo failed.", K(ret)); - } else if (OB_FAIL(mmw.append_table_lock_kv(mem_ctx_->get_max_table_version(), - redo))) { - if (OB_BUF_NOT_ENOUGH != ret) { - TRANS_LOG(WARN, "fill table lock redo fail", K(ret)); - } - } - - TRANS_LOG(DEBUG, "fill table lock redo.", - K(ret), K(*titer), K(redo.lock_id_), K(redo.lock_mode_)); - return ret; -} - int ObRedoLogGenerator::search_unsubmitted_dup_tablet_redo() { // OB_ENTRY_NOT_EXIST => no dup table tablet // OB_SUCCESS => find a dup table tablet int ret = OB_ENTRY_NOT_EXIST; - - ObITransCallbackIterator cursor; if (!is_inited_) { TRANS_LOG(WARN, "redo log generate is not inited", K(ret)); } else { - ObTransCallbackMgr::RDLockGuard guard(callback_mgr_->get_rwlock()); - for (cursor = generate_cursor_ + 1; OB_SUCC(ret) && callback_mgr_->end() != cursor; ++cursor) { - ObITransCallback *iter = (ObITransCallback *)*cursor; - - if (!iter->need_fill_redo() || !iter->need_submit_log()) { - //do nothing - } else if (check_dup_tablet_(iter)) { - // ret = OB_SUCCESS; - // mem_ctx_->get_trans_ctx()->set_dup_table_tx_(); - // break; - } + struct CheckDupTabletFunc final : public ObITxCallbackFinder { + bool match(ObITransCallback *callback) { + bool ok =false; + if (!callback->need_submit_log()) { + //do nothing + } else if (generator_->check_dup_tablet(callback)) { + ok = true; + } + return ok; + }; + ObRedoLogGenerator *generator_; + }; + CheckDupTabletFunc check_func; + check_func.generator_ = this; + if (callback_mgr_->find(check_func)) { + ret = OB_SUCCESS; + mem_ctx_->get_trans_ctx()->set_dup_table_tx_(); } } return ret; } -bool ObRedoLogGenerator::check_dup_tablet_(const ObITransCallback *callback_ptr) const +bool ObRedoLogGenerator::check_dup_tablet(const ObITransCallback *callback_ptr) const { bool is_dup_tablet = false; int64_t tmp_ret = OB_SUCCESS; @@ -431,18 +390,7 @@ bool ObRedoLogGenerator::check_dup_tablet_(const ObITransCallback *callback_ptr) void ObRedoLogGenerator::bug_detect_for_logging_blocked_() { - int ret = OB_SUCCESS; - - ObITransCallbackIterator bug_detect_cursor; - int64_t count = 0; - for (bug_detect_cursor = generate_cursor_ + 1; - callback_mgr_->end() !=bug_detect_cursor - && count <= 5; - ++bug_detect_cursor) { - ObITransCallback *bug_detect_iter = (ObITransCallback *)*bug_detect_cursor; - count++; - TRANS_LOG(WARN, "logging block print callback", KPC(bug_detect_iter), K(count)); - } + // TODO: (yunxing.cyx) print first 5 callback ? } }; // end namespace memtable diff --git a/src/storage/memtable/ob_redo_log_generator.h b/src/storage/memtable/ob_redo_log_generator.h index 9ae52de9f1..4259b54ef5 100644 --- a/src/storage/memtable/ob_redo_log_generator.h +++ b/src/storage/memtable/ob_redo_log_generator.h @@ -24,31 +24,127 @@ namespace memtable // Represents the callbacks in [start_, end_] struct ObCallbackScope { - ObCallbackScope() : start_(nullptr), end_(nullptr) {} + ObCallbackScope() : start_(nullptr), end_(nullptr), host_(nullptr), cnt_(0), data_size_(0) {} ~ObCallbackScope() {} void reset() { start_.reset(); end_.reset(); + host_ = nullptr; + cnt_ = 0; + data_size_ = 0; } bool is_empty() const { return (nullptr == *start_) || (nullptr == *end_); } ObITransCallbackIterator start_; ObITransCallbackIterator end_; + ObTxCallbackList *host_; + int32_t cnt_; + int64_t data_size_; + TO_STRING_KV("start", OB_P(*start_), "end", OB_P(*end_), K_(cnt), K_(data_size), KP_(host)); }; - +typedef ObIArray ObCallbackScopeArray; struct ObRedoLogSubmitHelper { - ObRedoLogSubmitHelper() : callbacks_(), max_seq_no_(), data_size_(0) {} + ObRedoLogSubmitHelper() : callbacks_(), max_seq_no_(), data_size_(0), callback_redo_submitted_(true) {} ~ObRedoLogSubmitHelper() {} void reset() { callbacks_.reset(); max_seq_no_.reset(); data_size_ = 0; + callback_redo_submitted_ = true; } - ObCallbackScope callbacks_; // callbacks in the redo log + ObSEArray callbacks_; // callbacks in the redo log transaction::ObTxSEQ max_seq_no_; int64_t data_size_; // records the data amount of all serialized trans node of this fill process + share::SCN log_scn_; + bool callback_redo_submitted_; +}; +struct RedoLogEpoch { + RedoLogEpoch() : v_(0) {} + RedoLogEpoch(int64_t v): v_(v) {} + operator int64_t&() { return v_; } + operator int64_t() const { return v_; } + DECLARE_TO_STRING + { + int64_t pos = 0; + if (v_ == INT64_MAX) { BUF_PRINTF("MAX"); } + else { BUF_PRINTF("%ld", v_); } + return pos; + } + int64_t v_; +}; + +struct ObTxFillRedoCtx +{ + ObTxFillRedoCtx() : + tx_id_(), + write_seq_no_(), + skip_lock_node_(false), + all_list_(false), + list_log_epoch_arr_(), + cur_epoch_(0), + next_epoch_(0), + epoch_from_(0), + epoch_to_(0), + list_(NULL), + list_idx_(-1), + callback_scope_(NULL), + buf_(NULL), + buf_len_(-1), + buf_pos_(-1), + helper_(NULL), + last_log_blocked_memtable_(NULL), + fill_count_(0), + is_all_filled_(false), + fill_time_(0) + { + list_log_epoch_arr_.set_max_print_count(256); + } + transaction::ObTransID tx_id_; + transaction::ObTxSEQ write_seq_no_; // to select callback list in parallel logging + bool skip_lock_node_; // whether skip fill lock node + bool all_list_; // whether to fill all callback-list + ObSEArray list_log_epoch_arr_; // record each list's next log epoch + int64_t cur_epoch_; // current filling epoch + int64_t next_epoch_; // next epoch of list, used to update list_log_epoch_arr_ + int64_t epoch_from_; // the epoch range to fill + int64_t epoch_to_; + ObTxCallbackList *list_; // current filling callback-list + int list_idx_; // fill from which list idx + ObCallbackScope *callback_scope_; // current filling callback scope + char* buf_; // the target buffer to fill + int64_t buf_len_; + int64_t buf_pos_; + ObRedoLogSubmitHelper *helper_; + ObMemtable *last_log_blocked_memtable_; + int fill_count_; // number of callbacks was filled + int fill_round_; // iter of `choice-list -> fill -> fill others` loop count + bool is_all_filled_; // no remains, all callbacks was filled + int64_t fill_time_; // time used +public: + bool is_empty() const { return fill_count_ == 0; } + bool not_empty() const { return fill_count_ > 0; } + TO_STRING_KV(K_(tx_id), K_(write_seq_no), K_(all_list), K_(cur_epoch), K_(next_epoch), + K_(epoch_from), K_(epoch_to), K_(fill_count), K_(fill_time), + KPC_(callback_scope), K_(skip_lock_node), K_(is_all_filled), K_(list_idx), + K_(list_log_epoch_arr), KP_(last_log_blocked_memtable), K_(buf_len), K_(buf_pos)); +}; + +class ObCallbackListLogGuard +{ +public: + ObCallbackListLogGuard() : lock_ptr_(NULL) {} + ~ObCallbackListLogGuard() { reset(); } + void reset() { + if (lock_ptr_) { + lock_ptr_->unlock(); + lock_ptr_ = NULL; + } + } + void set(common::ObByteLock *lock) { lock_ptr_ = lock; } +private: + common::ObByteLock *lock_ptr_; }; class ObRedoLogGenerator @@ -59,7 +155,6 @@ public: redo_filled_cnt_(0), redo_sync_succ_cnt_(0), redo_sync_fail_cnt_(0), - generate_cursor_(), callback_mgr_(nullptr), mem_ctx_(NULL), clog_encrypt_meta_(NULL) @@ -73,35 +168,22 @@ public: } void reset(); void reuse(); - int set(ObTransCallbackMgr *mgr, ObIMemtableCtx *mem_ctx); - int fill_redo_log(char *buf, - const int64_t buf_len, - int64_t &buf_pos, - ObRedoLogSubmitHelper &helper, - const bool log_for_lock_node); + int set(ObTransCallbackMgr *mgr, ObMemtableCtx *mem_ctx); + int fill_redo_log(ObTxFillRedoCtx &ctx); int search_unsubmitted_dup_tablet_redo(); - int log_submitted(const ObCallbackScope &callbacks); - int sync_log_succ(const share::SCN scn, const ObCallbackScope &callbacks); - void sync_log_fail(const ObCallbackScope &callbacks); - ObITransCallback *get_generate_cursor() { return (ObITransCallback *)*generate_cursor_; } - + int log_submitted(const ObCallbackScopeArray &callbacks, const share::SCN &scn); + int sync_log_succ(const ObCallbackScopeArray &callbacks, const share::SCN &scn); + void sync_log_fail(const ObCallbackScopeArray &callbacks, const share::SCN &scn); + void inc_sync_log_fail_cnt(const int cnt) + { + redo_sync_fail_cnt_ += cnt; + } int64_t get_redo_filled_count() const { return redo_filled_cnt_; } int64_t get_redo_sync_succ_count() const { return redo_sync_succ_cnt_; } int64_t get_redo_sync_fail_count() const { return redo_sync_fail_cnt_; } void print_first_mvcc_callback(); + bool check_dup_tablet(const ObITransCallback *callback_ptr) const; private: - int fill_row_redo(ObITransCallbackIterator &cursor, - ObMutatorWriter &mmw, - RedoDataNode &redo, - const bool log_for_lock_node, - bool &fake_fill, - transaction::ObCLogEncryptInfo &encrypt_info); - int fill_table_lock_redo(ObITransCallbackIterator &cursor, - ObMutatorWriter &mmw, - TableLockRedoDataNode &redo, - const bool log_for_lock_node, - bool &fake_fill); - bool check_dup_tablet_(const ObITransCallback * callback_ptr) const; void bug_detect_for_logging_blocked_(); private: DISALLOW_COPY_AND_ASSIGN(ObRedoLogGenerator); @@ -109,14 +191,14 @@ private: int64_t redo_filled_cnt_; int64_t redo_sync_succ_cnt_; int64_t redo_sync_fail_cnt_; - ObITransCallbackIterator generate_cursor_; // the pos of callback which already generated log ObTransCallbackMgr *callback_mgr_; - ObIMemtableCtx *mem_ctx_; + ObMemtableCtx *mem_ctx_; transaction::ObTxEncryptMeta *clog_encrypt_meta_; // logging block bug detector int64_t last_logging_blocked_time_; }; + }; // end namespace memtable }; // end namespace oceanbase diff --git a/src/storage/memtable/ob_row_compactor.cpp b/src/storage/memtable/ob_row_compactor.cpp index 85b449d26f..e0f0b1102a 100644 --- a/src/storage/memtable/ob_row_compactor.cpp +++ b/src/storage/memtable/ob_row_compactor.cpp @@ -180,8 +180,12 @@ int ObMemtableRowCompactor::try_cleanout_tx_node_during_compact_(ObTxTableGuard // So the case before will not happen again before v5 will not be delay // cleanout and filled back through commit callback. So we add the error // log back - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "unexpected non cleanout uncommitted node", KPC(tnode), KPC(row_)); + // + // since 4.3 multiple callback list will not merged, in commiting phase + // if two TransNode of same row stay in different callback list, their + // fill back order is undefined, and this situation can happened + // ret = OB_ERR_UNEXPECTED; + // TRANS_LOG(ERROR, "unexpected non cleanout uncommitted node", KPC(tnode), KPC(row_)); } else if (OB_FAIL(tx_table_guard.cleanout_tx_node(tnode->tx_id_, *row_, *tnode, false /*need_row_latch*/))) { TRANS_LOG(WARN, "cleanout tx state failed", K(ret), KPC(row_), KPC(tnode)); } @@ -219,6 +223,7 @@ ObMvccTransNode *ObMemtableRowCompactor::construct_compact_node_(const SCN snaps TRANS_LOG(DEBUG, "chaser debug start compact memtable row", K(memtable_->get_key())); // Scan nodes till tail OR a previous compact node OR a delete node. + bool giveup_compaction = false; while (OB_SUCCESS == ret && NULL != cur) { // Read cells & compact them by a map. const ObMemtableDataHeader *mtd = NULL; @@ -226,8 +231,15 @@ ObMvccTransNode *ObMemtableRowCompactor::construct_compact_node_(const SCN snaps if (OB_FAIL(try_cleanout_tx_node_during_compact_(tx_table_guard, cur))) { TRANS_LOG(WARN, "cleanout tx state failed", K(ret), KPC(row_), KPC(cur)); } else if (!(cur->is_aborted() || cur->is_committed() || cur->is_elr())) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "unexpected cleanout state", K(*cur), K(*row_)); + ObMvccTransNode *next = cur->next_; + if (next && (next->is_aborted() || next->is_committed() || next->is_elr())) { + // for safety, just giveup the compaction + giveup_compaction = true; + ret = OB_ITER_END; + } else { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "unexpected cleanout state", K(snapshot_version), KP(next), K(*cur), K(*row_)); + } } else if (cur->is_aborted()) { TRANS_LOG(INFO, "ignore aborted node when compact", K(*cur), K(*row_)); cur = cur->prev_; @@ -310,7 +322,7 @@ ObMvccTransNode *ObMemtableRowCompactor::construct_compact_node_(const SCN snaps ret = (OB_ITER_END == ret) ? OB_SUCCESS : ret; // Write compact row - if (OB_SUCC(ret) && compact_row_cnt > 0) { + if (OB_SUCC(ret) && !giveup_compaction && compact_row_cnt > 0) { EVENT_INC(MEMSTORE_ROW_COMPACTION_COUNT); SMART_VAR(blocksstable::ObRowWriter, row_writer) { char *buf = nullptr; diff --git a/src/storage/ob_i_store.cpp b/src/storage/ob_i_store.cpp index 8c6f3ca35b..f27aa7bff6 100644 --- a/src/storage/ob_i_store.cpp +++ b/src/storage/ob_i_store.cpp @@ -69,13 +69,13 @@ void ObStoreCtx::reset() { ls_id_.reset(); ls_ = nullptr; + branch_ = 0; tablet_id_.reset(); table_iter_ = nullptr; table_version_ = INT64_MAX; timeout_ = -1; mvcc_acc_ctx_.reset(); tablet_stat_.reset(); - replay_log_scn_.set_max(); is_read_store_ctx_ = false; } diff --git a/src/storage/ob_i_store.h b/src/storage/ob_i_store.h index 390474e233..2c692fcc0b 100644 --- a/src/storage/ob_i_store.h +++ b/src/storage/ob_i_store.h @@ -427,23 +427,23 @@ struct ObStoreCtx TO_STRING_KV(KP(this), K_(ls_id), KP_(ls), + K_(branch), K_(timeout), K_(tablet_id), KP_(table_iter), K_(table_version), K_(mvcc_acc_ctx), K_(tablet_stat), - K_(replay_log_scn), K_(is_read_store_ctx)); share::ObLSID ls_id_; storage::ObLS *ls_; // for performance opt + int16_t branch_; // parallel write id common::ObTabletID tablet_id_; mutable ObTableStoreIterator *table_iter_; int64_t table_version_; // used to update memtable's max_schema_version int64_t timeout_; memtable::ObMvccAccessCtx mvcc_acc_ctx_; // all txn relative context storage::ObTabletStat tablet_stat_; // used for collecting query statistics - share::SCN replay_log_scn_; // used in replay pass log_ts bool is_read_store_ctx_; }; diff --git a/src/storage/tablelock/ob_lock_memtable.cpp b/src/storage/tablelock/ob_lock_memtable.cpp index 6e5ed4427f..648cc0789f 100644 --- a/src/storage/tablelock/ob_lock_memtable.cpp +++ b/src/storage/tablelock/ob_lock_memtable.cpp @@ -154,7 +154,7 @@ int ObLockMemtable::lock_( lock_exist = false; lock_mode_in_same_trans = 0x0; conflict_tx_set.reset(); - ObMvccWriteGuard guard(true); + ObMvccWriteGuard guard; if (ObClockGenerator::getClock() >= param.expired_time_) { ret = (ret == OB_TRY_LOCK_ROW_CONFLICT ? OB_ERR_EXCLUSIVE_LOCK_CONFLICT : OB_TIMEOUT); LOG_WARN("lock timeout", K(ret), K(lock_op), K(param)); @@ -162,35 +162,39 @@ int ObLockMemtable::lock_( LOG_WARN("not allow lock table.", K(ret), K(ctx)); } else if (OB_FAIL(check_tablet_write_allow_(lock_op))) { LOG_WARN("check tablet write allow failed", K(ret), K(lock_op)); - } else if (FALSE_IT(mem_ctx = static_cast(ctx.mvcc_acc_ctx_.mem_ctx_))) { - } else if (OB_FAIL(mem_ctx->check_lock_exist(lock_op.lock_id_, - lock_op.owner_id_, - lock_op.lock_mode_, - lock_op.op_type_, - lock_exist, - lock_mode_in_same_trans))) { - LOG_WARN("failed to check lock exist ", K(ret), K(lock_op)); - } else if (lock_exist) { - // if the lock is DBMS_LOCK, we should return error code - // to notify PL to return the actual execution result. - if (lock_op.lock_id_.obj_type_ == ObLockOBJType::OBJ_TYPE_DBMS_LOCK) { - ret = OB_OBJ_LOCK_EXIST; + } else { + mem_ctx = static_cast(ctx.mvcc_acc_ctx_.mem_ctx_); + ObLockMemCtx::AddLockGuard guard(mem_ctx->get_lock_mem_ctx()); + if (OB_FAIL(guard.ret())) { + LOG_WARN("failed to acquire lock on lock_mem_ctx", K(ret), K(ctx)); + } else if (OB_FAIL(mem_ctx->check_lock_exist(lock_op.lock_id_, + lock_op.owner_id_, + lock_op.lock_mode_, + lock_op.op_type_, + lock_exist, + lock_mode_in_same_trans))) { + LOG_WARN("failed to check lock exist ", K(ret), K(lock_op)); + } else if (lock_exist) { + // if the lock is DBMS_LOCK, we should return error code + // to notify PL to return the actual execution result. + if (lock_op.lock_id_.obj_type_ == ObLockOBJType::OBJ_TYPE_DBMS_LOCK) { + ret = OB_OBJ_LOCK_EXIST; + } + LOG_DEBUG("lock is exist", K(ret), K(lock_op)); + } else if (FALSE_IT(lock_upgrade(lock_mode_in_same_trans, lock_op))) { + } else if (OB_FAIL(obj_lock_map_.lock(param, ctx, lock_op, lock_mode_in_same_trans, conflict_tx_set))) { + if (ret != OB_TRY_LOCK_ROW_CONFLICT && + ret != OB_OBJ_LOCK_EXIST) { + LOG_WARN("record lock at lock map mgr failed.", K(ret), K(lock_op)); + } + } else if (FALSE_IT(succ_step = STEP_IN_LOCK_MGR)) { + } else if (OB_FAIL(mem_ctx->add_lock_record(lock_op))) { + if (OB_EAGAIN == ret) { + need_retry = true; + } + LOG_WARN("record lock at mem_ctx failed.", K(ret), K(lock_op)); } - LOG_DEBUG("lock is exist", K(ret), K(lock_op)); - } else if (FALSE_IT(lock_upgrade(lock_mode_in_same_trans, lock_op))) { - } else if (OB_FAIL(obj_lock_map_.lock(param, ctx, lock_op, lock_mode_in_same_trans, conflict_tx_set))) { - if (ret != OB_TRY_LOCK_ROW_CONFLICT && - ret != OB_OBJ_LOCK_EXIST) { - LOG_WARN("record lock at lock map mgr failed.", K(ret), K(lock_op)); - } - } else if (FALSE_IT(succ_step = STEP_IN_LOCK_MGR)) { - } else if (OB_FAIL(mem_ctx->add_lock_record(lock_op))) { - if (OB_EAGAIN == ret) { - need_retry = true; - } - LOG_WARN("record lock at mem_ctx failed.", K(ret), K(lock_op)); } - if (OB_FAIL(ret) && succ_step == STEP_IN_LOCK_MGR) { obj_lock_map_.remove_lock_record(lock_op); } @@ -961,6 +965,7 @@ int ObLockMemtable::flush(SCN recycle_scn, int ObLockMemtable::replay_row( storage::ObStoreCtx &ctx, + const share::SCN &scn, ObMemtableMutatorIterator *mmi) { int ret = OB_SUCCESS; @@ -1005,7 +1010,7 @@ int ObLockMemtable::replay_row( if (OB_UNLIKELY(!lock_op.is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("lock op is not valid", K(ret), K(lock_op)); - } else if (OB_FAIL(replay_lock_(mem_ctx, lock_op, ctx.replay_log_scn_))) { + } else if (OB_FAIL(replay_lock_(mem_ctx, lock_op, scn))) { LOG_WARN("replay lock failed", K(ret), K(lock_op)); } } diff --git a/src/storage/tablelock/ob_lock_memtable.h b/src/storage/tablelock/ob_lock_memtable.h index a4f08e24ea..2a775a6432 100644 --- a/src/storage/tablelock/ob_lock_memtable.h +++ b/src/storage/tablelock/ob_lock_memtable.h @@ -134,6 +134,7 @@ public: // ====================== REPLAY LOCK ====================== virtual int replay_row(storage::ObStoreCtx &ctx, + const share::SCN &scn, memtable::ObMemtableMutatorIterator *mmi); // replay lock to lock map and trans part ctx. // used by the replay process of multi data source. diff --git a/src/storage/tablelock/ob_lock_table.cpp b/src/storage/tablelock/ob_lock_table.cpp index 56db8cc88a..840305d550 100644 --- a/src/storage/tablelock/ob_lock_table.cpp +++ b/src/storage/tablelock/ob_lock_table.cpp @@ -524,10 +524,11 @@ int ObLockTable::lock( ret = OB_ERR_UNEXPECTED; LOG_ERROR("invalid argument", K(ret), K(ctx), K(param), K(ctx.mvcc_acc_ctx_)); ob_abort(); - } else if (OB_FAIL(get_lock_memtable(handle))) { + } else if (OB_FAIL(ctx.mvcc_acc_ctx_.mem_ctx_->get_lock_mem_ctx().get_lock_memtable(memtable))) { LOG_WARN("get lock memtable failed", K(ret)); - } else if (OB_FAIL(handle.get_lock_memtable(memtable))) { - LOG_ERROR("get lock memtable from lock handle failed", K(ret)); + } else if (OB_ISNULL(memtable)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("lock memtable is null", K(ret), K(ctx)); } else { const int64_t lock_timestamp = ObTimeUtility::current_time(); ObTableLockOp lock_op(param.lock_id_, diff --git a/src/storage/tablelock/ob_mem_ctx_table_lock.cpp b/src/storage/tablelock/ob_mem_ctx_table_lock.cpp index d3bfc67940..aa31fded69 100644 --- a/src/storage/tablelock/ob_mem_ctx_table_lock.cpp +++ b/src/storage/tablelock/ob_mem_ctx_table_lock.cpp @@ -78,7 +78,7 @@ void ObLockMemCtx::reset() callback_pool_.reset(); } -void ObLockMemCtx::rollback_table_lock_(const ObTxSEQ seq_no) +int ObLockMemCtx::rollback_table_lock_(const ObTxSEQ to_seq_no, const ObTxSEQ from_seq_no) { int ret = OB_SUCCESS; ObLockMemtable *memtable = nullptr; @@ -86,8 +86,12 @@ void ObLockMemCtx::rollback_table_lock_(const ObTxSEQ seq_no) LOG_ERROR("get lock memtable failed", K(ret)); } else { DLIST_FOREACH_REMOVESAFE_NORET(curr, lock_list_) { - if (curr->lock_op_.lock_seq_no_ <= seq_no) { - // do nothing + if (curr->lock_op_.lock_seq_no_ <= to_seq_no || + curr->lock_op_.lock_seq_no_ > from_seq_no) { + // out of scope, do nothing + } else if (to_seq_no.get_branch() !=0 && + curr->lock_op_.lock_seq_no_.get_branch() != to_seq_no.get_branch()) { + // branch missmatch } else { memtable->remove_lock_record(curr->lock_op_); (void)lock_list_.remove(curr); @@ -96,6 +100,7 @@ void ObLockMemCtx::rollback_table_lock_(const ObTxSEQ seq_no) } } } + return ret; } void ObLockMemCtx::abort_table_lock_() @@ -153,7 +158,7 @@ int ObLockMemCtx::commit_table_lock_(const SCN &commit_version, const SCN &commi return ret; } -int ObLockMemCtx::rollback_table_lock(const ObTxSEQ seq_no) +int ObLockMemCtx::rollback_table_lock(const ObTxSEQ to_seq_no, const ObTxSEQ from_seq_no) { int ret = OB_SUCCESS; if (lock_list_.is_empty()) { @@ -163,9 +168,24 @@ int ObLockMemCtx::rollback_table_lock(const ObTxSEQ seq_no) LOG_WARN("memtable should not be null", K(ret), K(memtable_handle_)); } else { WRLockGuard guard(list_rwlock_); - rollback_table_lock_(seq_no); + if (OB_FAIL(rollback_table_lock_(to_seq_no, from_seq_no))) { + LOG_WARN("rollback table lock failed", K(ret), K(to_seq_no), K(from_seq_no)); + } + } + LOG_DEBUG("ObLockMemCtx::rollback_table_lock ", K(ret), K(to_seq_no), K(from_seq_no)); + return ret; +} + +int ObLockMemCtx::sync_log_succ(const share::SCN &scn) +{ + int ret = OB_SUCCESS; + // NOTE: the callback of sync log succ is ensured in asc order of SCN by TxCtx + if (max_durable_scn_ > scn) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("sync log succ is backoff", K(max_durable_scn_), K(scn)); + } else { + max_durable_scn_ = scn; } - LOG_DEBUG("ObLockMemCtx::rollback_table_lock ", K(ret), K(seq_no)); return ret; } @@ -174,10 +194,9 @@ int ObLockMemCtx::get_table_lock_store_info(ObTableLockInfo &table_lock_info) int ret = OB_SUCCESS; RDLockGuard guard(list_rwlock_); DLIST_FOREACH(curr, lock_list_) { - if (OB_UNLIKELY(!curr->is_valid()) || !curr->is_logged()) { + if (OB_UNLIKELY(!curr->is_valid())) { // no need dump to avoid been restored even if rollback - LOG_WARN("the table lock op no should not dump", - K(curr->lock_op_), K(curr->is_logged())); + LOG_WARN("the table lock op no should not dump", K(curr->lock_op_)); } else if (OB_FAIL(table_lock_info.table_lock_ops_.push_back(curr->lock_op_))) { LOG_WARN("fail to push back table_lock store info", K(ret)); break; @@ -205,15 +224,9 @@ int ObLockMemCtx::get_table_lock_for_transfer(ObTableLockInfo &table_lock_info, } if (OB_FAIL(ret)) { } else if (!is_hit) { - } else { - // table lock MDS not impl on_redo phase - if (!curr->is_logged()) { - LOG_WARN("lock op is not logged", K(*curr), K(tablet_list)); - } - if (OB_FAIL(table_lock_info.table_lock_ops_.push_back(curr->lock_op_))) { - LOG_WARN("fail to push back table_lock store info", K(ret)); - break; - } + } else if (OB_FAIL(table_lock_info.table_lock_ops_.push_back(curr->lock_op_))) { + LOG_WARN("fail to push back table_lock store info", K(ret)); + break; } } } @@ -247,8 +260,7 @@ int ObLockMemCtx::clear_table_lock( int ObLockMemCtx::add_lock_record( const ObTableLockOp &lock_op, - ObMemCtxLockOpLinkNode *&lock_op_node, - const bool logged) + ObMemCtxLockOpLinkNode *&lock_op_node) { int ret = OB_SUCCESS; void *ptr = NULL; @@ -268,8 +280,6 @@ int ObLockMemCtx::add_lock_record( if (!lock_list_.add_last(lock_op_node)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("add lock op info failed.", K(ret), K(lock_op)); - } else if (logged) { - lock_op_node->set_logged(); } } if (OB_FAIL(ret) && NULL != lock_op_node) { @@ -317,19 +327,6 @@ void ObLockMemCtx::remove_lock_record( LOG_DEBUG("ObLockMemCtx::remove_lock_record ", K(lock_op)); } -void ObLockMemCtx::set_log_synced( - ObMemCtxLockOpLinkNode *lock_op, - const SCN &scn) -{ - if (OB_ISNULL(lock_op)) { - LOG_WARN_RET(OB_INVALID_ARGUMENT, "invalid argument.", K(lock_op)); - } else { - max_durable_scn_.inc_update(scn); - lock_op->logged_ = true; - LOG_DEBUG("ObLockMemCtx::set_log_synced ", KPC(lock_op), K(scn)); - } -} - int ObLockMemCtx::check_lock_exist( //TODO(lihongqin):check it const ObLockID &lock_id, const ObTableLockOwnerID &owner_id, diff --git a/src/storage/tablelock/ob_mem_ctx_table_lock.h b/src/storage/tablelock/ob_mem_ctx_table_lock.h index fa68f17dd1..67fc56fae0 100644 --- a/src/storage/tablelock/ob_mem_ctx_table_lock.h +++ b/src/storage/tablelock/ob_mem_ctx_table_lock.h @@ -32,19 +32,13 @@ class ObMemCtxLockOpLinkNode : public common::ObDLinkBase ObLockNodeList; @@ -62,18 +56,16 @@ public: lock_list_(), is_killed_(false), max_durable_scn_(), - memtable_handle_() {} + memtable_handle_(), + add_lock_latch_() {} ObLockMemCtx() = delete; ~ObLockMemCtx() { reset(); } int init(storage::ObTableHandleV2 &handle); int get_lock_memtable(ObLockMemtable *&memtable); void reset(); - void set_log_synced(ObMemCtxLockOpLinkNode *lock_op, const share::SCN &scn); - int add_lock_record( const ObTableLockOp &lock_op, - ObMemCtxLockOpLinkNode *&lock_op_node, - const bool logged = false); + ObMemCtxLockOpLinkNode *&lock_op_node); void remove_lock_record( const ObTableLockOp &lock_op); void remove_lock_record( @@ -102,9 +94,10 @@ public: const bool is_committed, const share::SCN &commit_version, const share::SCN &commit_scn); - int rollback_table_lock(const ObTxSEQ seq_no); + int rollback_table_lock(const ObTxSEQ to_seq_no, const ObTxSEQ from_seq_no); void *alloc_lock_op_callback(); void free_lock_op_callback(void *cb); + int sync_log_succ(const share::SCN &scn); int get_table_lock_store_info(ObTableLockInfo &table_lock_info); int get_table_lock_for_transfer(ObTableLockInfo &table_lock_info, const ObIArray &tablet_list); // used by deadlock detector to kill the trans. @@ -113,12 +106,35 @@ public: // used to check whether the tx is killed by deadlock detector. bool is_killed() const { return is_killed_; } + +public: + class AddLockGuard + { + // use to serialize multi thread try to add one lock for same transaction + public: + AddLockGuard(ObLockMemCtx &ctx): ctx_(NULL) + { + if (OB_SUCCESS == (ret_ = ctx.add_lock_latch_.lock())) { + ctx_ = &ctx; + } + } + ~AddLockGuard() + { + if (ctx_) { + ctx_->add_lock_latch_.unlock(); + } + } + int ret() const { return ret_; } + private: + int ret_; + ObLockMemCtx *ctx_; + }; private: void *alloc_lock_op(); void free_lock_op(void *op); void free_lock_op_(void *op); void print() const; - void rollback_table_lock_(const ObTxSEQ seq_no); + int rollback_table_lock_(const ObTxSEQ to_seq_no, const ObTxSEQ from_seq_no); int commit_table_lock_(const share::SCN &commit_version, const share::SCN &commit_scn); void abort_table_lock_(); private: @@ -135,6 +151,9 @@ private: share::SCN max_durable_scn_; // the lock memtable pointer point to LS lock table's memtable. storage::ObTableHandleV2 memtable_handle_; +protected: + // serialze multiple thread try add lock for same transaction + ObSpinLock add_lock_latch_; }; } diff --git a/src/storage/tablelock/ob_obj_lock.cpp b/src/storage/tablelock/ob_obj_lock.cpp index 34bf2ffe87..ff02ea4acd 100644 --- a/src/storage/tablelock/ob_obj_lock.cpp +++ b/src/storage/tablelock/ob_obj_lock.cpp @@ -1781,7 +1781,7 @@ int ObOBJLockMap::lock( lock_map_.revert(obj_lock); } if (OB_FAIL(ret) && REACH_TIME_INTERVAL(1 * 1000 * 1000)) { - LOG_WARN("ObOBJLockMap::lock ", K(ret), K(param), K(lock_op)); + LOG_WARN("ObOBJLockMap::lock ", K(ret), K(param), K(lock_op), K(conflict_tx_set)); } // retry if the table lock list map is delete right now by others. } while (ret == OB_EAGAIN); diff --git a/src/storage/tablelock/ob_table_lock_callback.cpp b/src/storage/tablelock/ob_table_lock_callback.cpp index 72ec85a550..adfe639b50 100644 --- a/src/storage/tablelock/ob_table_lock_callback.cpp +++ b/src/storage/tablelock/ob_table_lock_callback.cpp @@ -40,26 +40,6 @@ memtable::ObIMemtable* ObOBJLockCallback::get_memtable() const return memtable_; } -int ObOBJLockCallback::log_sync(const SCN scn) -{ - int ret = OB_SUCCESS; - ObMemtableCtx *mem_ctx = static_cast(ctx_); - if (OB_UNLIKELY(SCN::max_scn() == scn)) { - ret = OB_INVALID_ARGUMENT; - LOG_ERROR("log ts should not be invalid", K(ret), K(scn), K(*this)); - } else if (OB_ISNULL(mem_ctx) || OB_ISNULL(lock_op_)) { - ret = OB_ERR_UNEXPECTED; - LOG_ERROR("unexpected error", K(ret), K(mem_ctx), K_(lock_op)); - } else { - // only version after 3.3 has table lock. - mem_ctx->update_max_submitted_seq_no(lock_op_->lock_op_.lock_seq_no_); - // TODO: yanyuan.cxf maybe need removed. - mem_ctx->set_log_synced(lock_op_, scn); - scn_ = scn; - } - return ret; -} - int ObOBJLockCallback::print_callback() { LOG_INFO("print callback", K(*this)); diff --git a/src/storage/tablelock/ob_table_lock_callback.h b/src/storage/tablelock/ob_table_lock_callback.h index f62525e428..a77a1c7b2d 100644 --- a/src/storage/tablelock/ob_table_lock_callback.h +++ b/src/storage/tablelock/ob_table_lock_callback.h @@ -63,9 +63,7 @@ public: virtual int del() override; transaction::ObTxSEQ get_seq_no() const override; bool is_table_lock_callback() const override { return true; } - bool log_synced() const override { return share::SCN::max_scn() != scn_; } bool must_log() const; - int log_sync(const share::SCN scn) override; int64_t get_data_size() override { return 0; } // size of trans node. memtable::MutatorType get_mutator_type() const override { return memtable::MutatorType::MUTATOR_TABLE_LOCK; } diff --git a/src/storage/tablelock/ob_table_lock_common.cpp b/src/storage/tablelock/ob_table_lock_common.cpp index 9f09880287..8e4c22f052 100644 --- a/src/storage/tablelock/ob_table_lock_common.cpp +++ b/src/storage/tablelock/ob_table_lock_common.cpp @@ -172,6 +172,11 @@ void ObTableLockOp::set( op_type_ = type; lock_op_status_ = lock_op_status; lock_seq_no_ = seq_no; + // here, ensure lock-callback was dispatched to single callback-list + // forcedly set the seq_no's branch to zero + if (lock_seq_no_.get_branch() != 0) { + lock_seq_no_.set_branch(0); + } create_timestamp_ = create_timestamp; create_schema_version_ = create_schema_version; } diff --git a/src/storage/tablelock/ob_table_lock_common.h b/src/storage/tablelock/ob_table_lock_common.h index 507fd1145a..a92a1bbeba 100644 --- a/src/storage/tablelock/ob_table_lock_common.h +++ b/src/storage/tablelock/ob_table_lock_common.h @@ -430,13 +430,17 @@ public: create_timestamp_(0), create_schema_version_(-1) { + // here, ensure lock-callback was dispatched to single callback-list + // forcedly set the seq_no's branch to zero + ObTxSEQ seq_no2 = seq_no; + seq_no2.set_branch(0); set(lock_id, lock_mode, owner_id, trans_id, op_type, lock_op_status, - seq_no, + seq_no2, create_timestamp, create_schema_version); } diff --git a/src/storage/tablet/ob_tablet.cpp b/src/storage/tablet/ob_tablet.cpp index d55923ebba..1666baaef2 100644 --- a/src/storage/tablet/ob_tablet.cpp +++ b/src/storage/tablet/ob_tablet.cpp @@ -3845,12 +3845,6 @@ int ObTablet::insert_rows( } } } - if (OB_SUCC(ret)) { - int tmp_ret = OB_SUCCESS; - if (OB_TMP_FAIL(store_ctx.mvcc_acc_ctx_.tx_ctx_->submit_redo_log(false))) { - TRANS_LOG(INFO, "Failed to submit log if necessary", K(tmp_ret), K(store_ctx), K(relative_table)); - } - } return ret; } diff --git a/src/storage/tablet/ob_tablet_memtable_mgr.cpp b/src/storage/tablet/ob_tablet_memtable_mgr.cpp index f8301bcfbc..2fb017aed0 100644 --- a/src/storage/tablet/ob_tablet_memtable_mgr.cpp +++ b/src/storage/tablet/ob_tablet_memtable_mgr.cpp @@ -166,6 +166,48 @@ int ObTabletMemtableMgr::reset_storage_recorder() return ret; } +inline int ObTabletMemtableMgr::try_resolve_boundary_on_create_memtable_( + memtable::ObMemtable *last_frozen_memtable, + memtable::ObMemtable *new_memtable) +{ + int ret = OB_SUCCESS; + bool double_check = false; + bool can_resolve = false; + int64_t write_ref = 0; + int64_t unsubmitted_cnt = 0; + do { + write_ref = last_frozen_memtable->get_write_ref(); + unsubmitted_cnt = last_frozen_memtable->get_unsubmitted_cnt(); + if (0 == write_ref && 0 == unsubmitted_cnt) { + share::SCN max_decided_scn; + if (OB_FAIL(freezer_->get_max_consequent_callbacked_scn(max_decided_scn))) { + TRANS_LOG(WARN, "get max decided scn fail", K(ret), K(freezer_->get_ls_id())); + } else if (max_decided_scn >= last_frozen_memtable->get_end_scn()) { + // logstream's continous apply has pass frozen memtable's right boundary + can_resolve = true; + } + } + if (!can_resolve && !double_check) { + last_frozen_memtable->set_resolved_active_memtable_left_boundary(false); + } + double_check = !double_check; + } while (!can_resolve && double_check); + + if (can_resolve) { + last_frozen_memtable->set_resolved_active_memtable_left_boundary(true); + last_frozen_memtable->resolve_right_boundary(); + TRANS_LOG(INFO, "[resolve_right_boundary] in create_memtable on leader", KPC(last_frozen_memtable)); + if (new_memtable != last_frozen_memtable) { + const SCN &new_start_scn = MAX(last_frozen_memtable->get_end_scn(), last_frozen_memtable->get_migration_clog_checkpoint_scn()); + new_memtable->resolve_left_boundary(new_start_scn); + } + } else if (unsubmitted_cnt > 0 || write_ref > 0) { + new_memtable->set_logging_blocked(); + TRANS_LOG(INFO, "set new memtable logging blocked", KPC(last_frozen_memtable), KPC(new_memtable)); + } + return ret; +} + // There are two cases: // 1. create the first memtable for tablet // 2. create the new memtable after freezing the old memtable @@ -258,33 +300,24 @@ int ObTabletMemtableMgr::create_memtable(const SCN clog_checkpoint_scn, if (OB_FAIL(ret)) { } else if (OB_NOT_NULL(last_frozen_memtable)) { - // keep the check order: is_frozen, write_ref_cnt, then unsubmitted_cnt and unsynced_cnt - int64_t write_ref = last_frozen_memtable->get_write_ref(); - int64_t unsubmitted_cnt = last_frozen_memtable->get_unsubmitted_cnt(); - int64_t unsynced_cnt = last_frozen_memtable->get_unsynced_cnt(); - if (write_ref > 0 || unsubmitted_cnt > 0) { - memtable->set_logging_blocked(); - TRANS_LOG(INFO, "set logging_block", KPC(last_frozen_memtable), KPC(memtable)); - } - if (write_ref > 0 || unsynced_cnt > 0) { - last_frozen_memtable->set_resolve_active_memtable_left_boundary(false); - } // for follower, must decide the boundary of frozen memtable - // for leader, decide the boundary of frozen memtable that meets ready_for_flush - if (for_replay || (0 == write_ref && - 0 == unsubmitted_cnt && - 0 == unsynced_cnt)) { + if (for_replay) { last_frozen_memtable->resolve_right_boundary(); - TRANS_LOG(INFO, "[resolve_right_boundary] last_frozen_memtable in create_memtable", K(for_replay), K(ls_id), KPC(last_frozen_memtable)); + TRANS_LOG(INFO, "[resolve_right_boundary] in create_memtable on replay", KPC(last_frozen_memtable)); if (memtable != last_frozen_memtable) { - const SCN &new_start_scn = MAX(last_frozen_memtable->get_end_scn(), last_frozen_memtable->get_migration_clog_checkpoint_scn()); - memtable->resolve_left_boundary(new_start_scn); + const SCN start_scn = MAX(last_frozen_memtable->get_end_scn(), + last_frozen_memtable->get_migration_clog_checkpoint_scn()); + memtable->resolve_left_boundary(start_scn); } } - // there is no frozen memtable and new sstable will not be generated, - // meaning that clog_checkpoint_scn will not be updated now, - // so get newest clog_checkpoint_scn to set left boundary + // for leader, decide the right boundary of frozen memtable + else if (OB_FAIL(try_resolve_boundary_on_create_memtable_(last_frozen_memtable, memtable))) { + TRANS_LOG(WARN, "try resolve boundary fail", K(ret)); + } } else { + // there is no frozen memtable and new sstable will not be generated, + // meaning that clog_checkpoint_scn will not be updated now, + // so get newest clog_checkpoint_scn to set left boundary memtable->resolve_left_boundary(new_clog_checkpoint_scn); } @@ -464,7 +497,9 @@ int ObTabletMemtableMgr::resolve_left_boundary_for_active_memtable(memtable::ObI ret = OB_NOT_INIT; LOG_WARN("not inited", K(ret), K_(is_inited)); } else if (OB_FAIL(get_active_memtable(handle))) { - LOG_WARN( "fail to get active memtable", K(ret)); + if (OB_ENTRY_NOT_EXIST != ret) { + LOG_WARN( "fail to get active memtable", K(ret)); + } } else if (OB_FAIL(handle.get_memtable(active_memtable))) { LOG_WARN("fail to get active memtable", K(ret)); } else { @@ -475,7 +510,7 @@ int ObTabletMemtableMgr::resolve_left_boundary_for_active_memtable(memtable::ObI ret = OB_SUCCESS; } if (OB_SUCC(ret)) { - static_cast(memtable)->set_resolve_active_memtable_left_boundary(true); + static_cast(memtable)->set_resolved_active_memtable_left_boundary(true); } return ret; diff --git a/src/storage/tablet/ob_tablet_memtable_mgr.h b/src/storage/tablet/ob_tablet_memtable_mgr.h index 422f47c935..11981aea37 100644 --- a/src/storage/tablet/ob_tablet_memtable_mgr.h +++ b/src/storage/tablet/ob_tablet_memtable_mgr.h @@ -126,6 +126,8 @@ private: int get_first_frozen_memtable_(ObTableHandleV2 &handle) const; void clean_tail_memtable_(); int get_last_frozen_memtable_(ObTableHandleV2 &handle) const; + int try_resolve_boundary_on_create_memtable_(memtable::ObMemtable *last_frozen_memtable, + memtable::ObMemtable *new_memtable); int resolve_left_boundary_for_active_memtable(memtable::ObIMemtable *memtable, share::SCN start_scn, share::SCN snapshot_version); diff --git a/src/storage/tx/ob_ctx_tx_data.cpp b/src/storage/tx/ob_ctx_tx_data.cpp index 09196acfd8..6761ee6156 100644 --- a/src/storage/tx/ob_ctx_tx_data.cpp +++ b/src/storage/tx/ob_ctx_tx_data.cpp @@ -68,6 +68,7 @@ void ObCtxTxData::reset() ctx_mgr_ = nullptr; tx_data_guard_.reset(); read_only_ = false; + recovered_from_tx_table_ = false; } void ObCtxTxData::destroy() @@ -98,27 +99,7 @@ int ObCtxTxData::insert_into_tx_table() return ret; } -int ObCtxTxData::recover_tx_data(ObTxDataGuard &rhs) -{ - int ret = OB_SUCCESS; - WLockGuard guard(lock_); - ObTxTable *tx_table = nullptr; - GET_TX_TABLE_(tx_table); - - if (OB_FAIL(ret)) { - } else if (OB_FAIL(check_tx_data_writable_())) { - TRANS_LOG(WARN, "tx data is not writeable", K(ret), KPC(this)); - } else if (OB_ISNULL(rhs.tx_data())) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "input tx data guard is unexpected nullptr", K(ret), KPC(this)); - } else if (OB_FAIL(tx_data_guard_.init(rhs.tx_data()))) { - TRANS_LOG(WARN, "init tx data guard failed", K(ret), KPC(this)); - } - - return ret; -} - -int ObCtxTxData::replace_tx_data(ObTxData *tmp_tx_data) +int ObCtxTxData::recover_tx_data(ObTxData *tmp_tx_data) { int ret = OB_SUCCESS; WLockGuard guard(lock_); @@ -135,6 +116,8 @@ int ObCtxTxData::replace_tx_data(ObTxData *tmp_tx_data) tx_data_guard_.reset(); if (OB_FAIL(tx_data_guard_.init(tmp_tx_data))) { TRANS_LOG(WARN, "init tx data guard failed", KR(ret), KPC(tmp_tx_data)); + } else { + recovered_from_tx_table_ = true; } } return ret; diff --git a/src/storage/tx/ob_ctx_tx_data.h b/src/storage/tx/ob_ctx_tx_data.h index 30618e47ce..d3318795e6 100644 --- a/src/storage/tx/ob_ctx_tx_data.h +++ b/src/storage/tx/ob_ctx_tx_data.h @@ -37,9 +37,9 @@ public: int init(const int64_t abs_expire_time, ObLSTxCtxMgr *ctx_mgr, int64_t tx_id); bool is_read_only() const { return read_only_; } + bool has_recovered_from_tx_table() const { return recovered_from_tx_table_; } int insert_into_tx_table(); - int recover_tx_data(storage::ObTxDataGuard &rhs); - int replace_tx_data(storage::ObTxData *tmp_tx_data); + int recover_tx_data(storage::ObTxData *tmp_tx_data); int deep_copy_tx_data_out(storage::ObTxDataGuard &tmp_tx_data_guard); // int alloc_tmp_tx_data(storage::ObTxDataGuard &tmp_tx_data); int free_tmp_tx_data(storage::ObTxData *&tmp_tx_data); @@ -108,6 +108,7 @@ private: ObLSTxCtxMgr *ctx_mgr_; storage::ObTxDataGuard tx_data_guard_; bool read_only_; + bool recovered_from_tx_table_; // lock for tx_data_ pointer RWLock lock_; }; diff --git a/src/storage/tx/ob_trans_ctx.cpp b/src/storage/tx/ob_trans_ctx.cpp index f399d394fe..061cefc13d 100644 --- a/src/storage/tx/ob_trans_ctx.cpp +++ b/src/storage/tx/ob_trans_ctx.cpp @@ -27,21 +27,14 @@ using namespace share; namespace transaction { -void ObTransCtx::get_ctx_guard(CtxLockGuard &guard) +void ObTransCtx::get_ctx_guard(CtxLockGuard &guard, uint8_t mode) { - guard.set(lock_); + guard.set(lock_, mode); } void ObTransCtx::print_trace_log() { - int ret = OB_SUCCESS; - if (OB_FAIL(lock_.try_lock())) { - TRANS_LOG(WARN, "print trace log trylock error", K(ret)); - } else { - print_trace_log_(); - lock_.unlock(); - } - UNUSED(ret); + print_trace_log_(); } void ObTransCtx::print_trace_log_() @@ -221,21 +214,6 @@ int ObTransCtx::defer_callback_scheduler_(const int retcode, const SCN &commit_v return ret; } -void ObTransCtx::test_lock(ObTxLogCb *log_cb) -{ - const int64_t before_lock_time = ObClockGenerator::getRealClock(); - CtxLockGuard guard(lock_); - const int64_t after_lock_time = ObClockGenerator::getRealClock(); - const int64_t log_sync_used_time = before_lock_time - log_cb->get_submit_ts(); - const int64_t ctx_lock_wait_time = after_lock_time - before_lock_time; - - if (log_sync_used_time + ctx_lock_wait_time - >= ObServerConfig::get_instance().clog_sync_time_warn_threshold) { - TRANS_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "transaction log sync use too much time", KPC(log_cb), - K(log_sync_used_time), K(ctx_lock_wait_time)); - } -} - void ObTransCtx::generate_request_id_() { const int64_t request_id = ObClockGenerator::getClock(); diff --git a/src/storage/tx/ob_trans_ctx.h b/src/storage/tx/ob_trans_ctx.h index bc1f23a6bf..14472bbba1 100644 --- a/src/storage/tx/ob_trans_ctx.h +++ b/src/storage/tx/ob_trans_ctx.h @@ -113,7 +113,7 @@ public: virtual ~ObTransCtx() { } void reset() { } public: - void get_ctx_guard(CtxLockGuard &guard); + void get_ctx_guard(CtxLockGuard &guard, uint8_t mode = CtxLockGuard::MODE::ALL); void print_trace_log(); // ATTENTION! There is no lock protect bool is_too_long_transaction() const @@ -163,7 +163,6 @@ public: int acquire_ctx_ref() { return acquire_ctx_ref_(); } ObITransRpc *get_trans_rpc() const { return rpc_; } - void test_lock(ObTxLogCb *log_cb); public: virtual bool is_inited() const = 0; virtual int handle_timeout(const int64_t delay) = 0; diff --git a/src/storage/tx/ob_trans_ctx_lock.cpp b/src/storage/tx/ob_trans_ctx_lock.cpp index 8471a701d8..afac5b4999 100644 --- a/src/storage/tx/ob_trans_ctx_lock.cpp +++ b/src/storage/tx/ob_trans_ctx_lock.cpp @@ -56,61 +56,164 @@ void CtxLock::after_unlock(CtxLockArg &arg) } } -int CtxLock::lock() -{ - ATOMIC_INC(&waiting_lock_cnt_); - int ret = lock_.wrlock(common::ObLatchIds::TRANS_CTX_LOCK); - ATOMIC_DEC(&waiting_lock_cnt_); - lock_start_ts_ = ObClockGenerator::getClock(); - return ret; -} +thread_local pid_t lock_thread_id_ = 0; +#define CHECK_LOCK(the_lock) \ + { \ + int64_t tid = - (OB_E(EventTable::EN_CHECK_TX_CTX_LOCK) OB_SUCCESS); \ + if (tid) { \ + if (lock_thread_id_ == 0) { \ + lock_thread_id_ = (pid_t)syscall(__NR_gettid); \ + } \ + const int64_t diff = tid - lock_thread_id_; \ + if (diff == 0) { \ + TRANS_LOG(INFO, "[CHECK_LOCK]", "lock", the_lock, \ + K(tid), K(lock_thread_id_), K(lbt())); \ + } \ + } \ + } int CtxLock::lock(const int64_t timeout_us) { - int ret = lock_.wrlock(common::ObLatchIds::TRANS_CTX_LOCK, ObTimeUtility::current_time() + timeout_us); - lock_start_ts_ = ObClockGenerator::getClock(); + ATOMIC_INC(&waiting_lock_cnt_); + int ret = OB_SUCCESS; + int64_t timeout = timeout_us >= 0 ? (ObTimeUtility::current_time() + timeout_us) : INT64_MAX; + if (OB_SUCC(access_lock_.wrlock(common::ObLatchIds::TRANS_ACCESS_LOCK, timeout))) { + CHECK_LOCK(access_lock_); + if (OB_SUCC(flush_redo_lock_.wrlock(common::ObLatchIds::TRANS_FLUSH_REDO_LOCK, timeout))) { + if (OB_SUCC(ctx_lock_.wrlock(common::ObLatchIds::TRANS_CTX_LOCK, timeout))) { + lock_start_ts_ = ObClockGenerator::getClock(); + } else { + flush_redo_lock_.unlock(); + access_lock_.unlock(); + } + } else { + access_lock_.unlock(); + } + } + ATOMIC_DEC(&waiting_lock_cnt_); return ret; } int CtxLock::try_lock() { - int ret = lock_.try_wrlock(common::ObLatchIds::TRANS_CTX_LOCK); - if (OB_SUCCESS == ret) { - lock_start_ts_ = ObClockGenerator::getClock(); + int ret = OB_SUCCESS; + if (OB_SUCC(access_lock_.try_wrlock(common::ObLatchIds::TRANS_ACCESS_LOCK))) { + CHECK_LOCK(access_lock_); + if (OB_SUCC(flush_redo_lock_.try_wrlock(common::ObLatchIds::TRANS_FLUSH_REDO_LOCK))) { + if (OB_SUCC(ctx_lock_.try_wrlock(common::ObLatchIds::TRANS_CTX_LOCK))) { + lock_start_ts_ = ObClockGenerator::getClock(); + } else { + flush_redo_lock_.unlock(); + access_lock_.unlock(); + } + } else { + access_lock_.unlock(); + } } return ret; } +int CtxLock::try_rdlock_ctx() +{ + int ret = ctx_lock_.try_rdlock(common::ObLatchIds::TRANS_CTX_LOCK); + lock_start_ts_ = 0; + return ret; +} + void CtxLock::unlock() { + unlock_access(); + unlock_flush_redo(); + unlock_ctx(); +} + +int CtxLock::wrlock_ctx() +{ + int ret = ctx_lock_.wrlock(common::ObLatchIds::TRANS_CTX_LOCK); + lock_start_ts_ = ObClockGenerator::getClock(); + return ret; +} + +int CtxLock::wrlock_access() +{ + CHECK_LOCK(access_lock_); + return access_lock_.wrlock(common::ObLatchIds::TRANS_ACCESS_LOCK); +} +void CtxLock::unlock_access() +{ + access_lock_.unlock(); +} +void CtxLock::unlock_ctx() +{ + const int64_t lock_start_ts = lock_start_ts_; CtxLockArg arg; before_unlock(arg); - lock_.unlock(); - const int64_t lock_ts = ObClockGenerator::getClock() - lock_start_ts_; - if (lock_start_ts_ > 0 && lock_ts > WARN_LOCK_TS) { - lock_start_ts_ = 0; - TRANS_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "ctx lock too much time", K(arg.trans_id_), K(lock_ts), K(lbt())); + ctx_lock_.unlock(); + if (lock_start_ts > 0) { + const int64_t lock_ts = ObClockGenerator::getClock() - lock_start_ts; + if (lock_ts > WARN_LOCK_TS) { + TRANS_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "ctx lock too much time", K(arg.trans_id_), K(lock_ts), K(lbt())); + } } after_unlock(arg); } +int CtxLock::wrlock_flush_redo() +{ + int ret = flush_redo_lock_.wrlock(common::ObLatchIds::TRANS_FLUSH_REDO_LOCK); + return ret; +} + +int CtxLock::rdlock_flush_redo() +{ + int ret = flush_redo_lock_.rdlock(common::ObLatchIds::TRANS_FLUSH_REDO_LOCK); + return ret; +} + +int CtxLock::try_rdlock_flush_redo() +{ + int ret = flush_redo_lock_.try_rdlock(common::ObLatchIds::TRANS_FLUSH_REDO_LOCK); + return ret; +} + +void CtxLock::unlock_flush_redo() +{ + flush_redo_lock_.unlock(); +} + CtxLockGuard::~CtxLockGuard() { reset(); } -void CtxLockGuard::set(CtxLock &lock) +void CtxLockGuard::set(CtxLock &lock, uint8_t mode) { reset(); lock_ = &lock; - lock_->lock(); + mode_ = mode; + do_lock_(true); } void CtxLockGuard::reset() { if (NULL != lock_) { - lock_->unlock(); + if (mode_ & MODE::CTX) { + lock_->unlock_ctx(); + } + if ((mode_ & MODE::REDO_FLUSH_X) || (mode_ & MODE::REDO_FLUSH_R)) { + lock_->unlock_flush_redo(); + } + if (mode_ & MODE::ACCESS) { + lock_->unlock_access(); + } + lock_ = NULL; + int64_t release_ts = ObTimeUtility::fast_current_time(); + if (release_ts - request_ts_ > 50_ms) { + TRANS_LOG_RET(WARN, OB_SUCCESS, "[slow ctx lock]", + "request_used", hold_ts_ - request_ts_, + "hold_used", release_ts - hold_ts_, K(lbt())); + } } } diff --git a/src/storage/tx/ob_trans_ctx_lock.h b/src/storage/tx/ob_trans_ctx_lock.h index c091a4af16..7c9530fdf0 100644 --- a/src/storage/tx/ob_trans_ctx_lock.h +++ b/src/storage/tx/ob_trans_ctx_lock.h @@ -57,24 +57,35 @@ public: class CtxLock { public: - CtxLock() : lock_(), ctx_(NULL), lock_start_ts_(0), waiting_lock_cnt_(0) {} +CtxLock() : ctx_lock_(), access_lock_(), flush_redo_lock_(), + ctx_(NULL), lock_start_ts_(0), waiting_lock_cnt_(0) {} ~CtxLock() {} int init(ObTransCtx *ctx); void reset(); - int lock(); - int lock(const int64_t timeout_us); + int lock(const int64_t timeout_us = -1); int try_lock(); void unlock(); + int try_rdlock_ctx(); + int wrlock_ctx(); + int wrlock_access(); + int wrlock_flush_redo(); + int rdlock_flush_redo(); + void unlock_ctx(); + void unlock_access(); + int try_rdlock_flush_redo(); + void unlock_flush_redo(); void before_unlock(CtxLockArg &arg); void after_unlock(CtxLockArg &arg); ObTransCtx *get_ctx() { return ctx_; } - bool is_locked_by_self() const { return lock_.is_wrlocked_by(); } + bool is_locked_by_self() const { return ctx_lock_.is_wrlocked_by(); } int64_t get_waiting_lock_cnt() const { return ATOMIC_LOAD(&waiting_lock_cnt_); } private: static const int64_t WARN_LOCK_TS = 1 * 1000 * 1000; DISALLOW_COPY_AND_ASSIGN(CtxLock); private: - common::ObLatch lock_; + common::ObLatch ctx_lock_; + common::ObLatch access_lock_; + common::ObLatch flush_redo_lock_; ObTransCtx *ctx_; int64_t lock_start_ts_; int64_t waiting_lock_cnt_; @@ -83,15 +94,51 @@ private: class CtxLockGuard { public: - CtxLockGuard() : lock_(NULL) {} - explicit CtxLockGuard(CtxLock &lock, const bool need_lock = true) : lock_(&lock) { if (need_lock) lock_->lock(); } + enum MODE { CTX = 1, ACCESS = 2, REDO_FLUSH_X = 4, REDO_FLUSH_R = 8, ALL = (CTX | REDO_FLUSH_X | ACCESS) }; + CtxLockGuard() : lock_(NULL), mode_(0), request_ts_(0), hold_ts_(0) {} + explicit CtxLockGuard(CtxLock &lock, int mode, bool need_lock = true): lock_(&lock), mode_(mode) + { do_lock_(need_lock); } + void do_lock_(bool need_lock) + { + request_ts_ = ObTimeUtility::fast_current_time(); + if (mode_ & ACCESS) { + if (need_lock) { + lock_->wrlock_access(); + } + } + if (mode_ & REDO_FLUSH_X) { + if (need_lock) { + lock_->wrlock_flush_redo(); + } + } + if (mode_ & REDO_FLUSH_R) { + if (need_lock) { + lock_->rdlock_flush_redo(); + } + } + if (mode_ & CTX) { + if (need_lock) { + lock_->wrlock_ctx(); + } + } + hold_ts_ = ObTimeUtility::fast_current_time(); + } + explicit CtxLockGuard(CtxLock &lock, const bool need_lock = true) + : CtxLockGuard(lock, MODE::ALL, need_lock) {} ~CtxLockGuard(); - void set(CtxLock &lock); + void set(CtxLock &lock, uint8_t mode = MODE::ALL); void reset(); + int64_t get_lock_acquire_used_time() const + { + return hold_ts_ - request_ts_; + } private: DISALLOW_COPY_AND_ASSIGN(CtxLockGuard); private: CtxLock *lock_; + uint8_t mode_; + int64_t request_ts_; + int64_t hold_ts_; }; diff --git a/src/storage/tx/ob_trans_define.cpp b/src/storage/tx/ob_trans_define.cpp index 2dfbb79778..a81d76a3ac 100644 --- a/src/storage/tx/ob_trans_define.cpp +++ b/src/storage/tx/ob_trans_define.cpp @@ -665,21 +665,6 @@ void ObTransNeedWaitWrap::set_trans_need_wait_wrap(const MonotonicTs receive_gts OB_SERIALIZE_MEMBER(ObUndoAction, undo_from_, undo_to_); -int ObUndoAction::merge(const ObUndoAction &other) -{ - int ret = OB_SUCCESS; - - if (!is_contain(other)) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid argument", K(*this), K(other)); - } else { - undo_from_ = MAX(undo_from_, other.undo_from_); - undo_to_ = MIN(undo_to_, other.undo_to_); - } - - return ret; -} - int ObEndParticipantsRes::add_blocked_trans_id(const ObTransID &trans_id) { return blocked_trans_ids_.push_back(trans_id); @@ -725,8 +710,10 @@ void ObTxExecInfo::reset() max_applying_log_ts_.reset(); max_applying_part_log_no_ = INT64_MAX; max_submitted_seq_no_.reset(); - checksum_ = 0; - checksum_scn_.set_min(); + checksum_.reset(); + checksum_.push_back(0); + checksum_scn_.reset(); + checksum_scn_.push_back(share::SCN::min_scn()); max_durable_lsn_.reset(); data_complete_ = false; is_dup_tx_ = false; @@ -739,6 +726,8 @@ void ObTxExecInfo::reset() is_transfer_blocking_ = false; is_empty_ctx_created_by_transfer_ = false; exec_epoch_ = 0; + serial_final_scn_.reset(); + serial_final_seq_no_.reset(); } void ObTxExecInfo::destroy(ObTxMDSCache &mds_cache) @@ -881,8 +870,11 @@ int ObTxExecInfo::assign(const ObTxExecInfo &exec_info) max_applying_log_ts_ = exec_info.max_applying_log_ts_; max_applying_part_log_no_ = exec_info.max_applying_part_log_no_; max_submitted_seq_no_ = exec_info.max_submitted_seq_no_; - checksum_ = exec_info.checksum_; - checksum_scn_ = exec_info.checksum_scn_; + if (OB_FAIL(checksum_.assign(exec_info.checksum_))) { + TRANS_LOG(WARN, "assign failed", K(ret)); + } else if (OB_FAIL(checksum_scn_.assign(exec_info.checksum_scn_))) { + TRANS_LOG(WARN, "assign failed", K(ret)); + } max_durable_lsn_ = exec_info.max_durable_lsn_; data_complete_ = exec_info.data_complete_; is_dup_tx_ = exec_info.is_dup_tx_; @@ -892,6 +884,8 @@ int ObTxExecInfo::assign(const ObTxExecInfo &exec_info) is_transfer_blocking_ = exec_info.is_transfer_blocking_; is_empty_ctx_created_by_transfer_ = exec_info.is_empty_ctx_created_by_transfer_; exec_epoch_ = exec_info.exec_epoch_; + serial_final_scn_ = exec_info.serial_final_scn_; + serial_final_seq_no_ = exec_info.serial_final_seq_no_; } return ret; } @@ -912,8 +906,8 @@ OB_SERIALIZE_MEMBER(ObTxExecInfo, max_applied_log_ts_, max_applying_part_log_no_, max_submitted_seq_no_, - checksum_, - checksum_scn_, + checksum_[0], + checksum_scn_[0], max_durable_lsn_, data_complete_, is_dup_tx_, @@ -928,7 +922,11 @@ OB_SERIALIZE_MEMBER(ObTxExecInfo, commit_parts_, transfer_parts_, is_empty_ctx_created_by_transfer_, - exec_epoch_); + exec_epoch_, + checksum_, + checksum_scn_, + serial_final_scn_, + serial_final_seq_no_); bool ObMulSourceDataNotifyArg::is_redo_submitted() const { return redo_submitted_; } diff --git a/src/storage/tx/ob_trans_define.h b/src/storage/tx/ob_trans_define.h index 51e41782d0..0608387f52 100644 --- a/src/storage/tx/ob_trans_define.h +++ b/src/storage/tx/ob_trans_define.h @@ -89,7 +89,7 @@ class ObXACtx; class ObITxCallback; class ObTxMultiDataSourceLog; typedef palf::LSN LogOffSet; - +enum { MAX_CALLBACK_LIST_COUNT = OB_MAX_CPU_NUM }; class ObTransErrsim { public: @@ -311,10 +311,10 @@ public: void reset() { raw_val_ = 0; } bool is_valid() const { return raw_val_ > 0; } bool is_max() const { return *this == MAX_VAL(); } - ObTxSEQ clone_with_seq(int64_t seq_n) const + ObTxSEQ clone_with_seq(int64_t seq_abs, int64_t seq_base) const { ObTxSEQ n = *this; - if (n_format_) { n.seq_ = seq_n; } else { n.seq_v0_ = seq_n; } + if (n_format_) { n.seq_ = seq_abs - seq_base; } else { n.seq_v0_ = seq_abs; } return n; } bool operator>(const ObTxSEQ &b) const @@ -323,7 +323,7 @@ public: } bool operator>=(const ObTxSEQ &b) const { - return *this > b || *this == b; + return *this > b || (n_format_ ? (seq_ == b.seq_) : (seq_v0_ == b.seq_v0_)); } bool operator<(const ObTxSEQ &b) const { @@ -354,9 +354,11 @@ public: int64_t inc_update(const ObTxSEQ &b) { return common::inc_update(&raw_val_, b.raw_val_); } uint64_t cast_to_int() const { return raw_val_; } static ObTxSEQ cast_from_int(int64_t seq) { return ObTxSEQ(seq); } + bool support_branch() const { return n_format_; } // return sequence number int64_t get_seq() const { return n_format_ ? seq_ : seq_v0_; } int16_t get_branch() const { return n_format_ ? branch_ : 0; } + ObTxSEQ &set_branch(int16_t branch) { branch_ = branch; return *this; } // atomic Load/Store void atomic_reset() { ATOMIC_SET(&raw_val_, 0); } ObTxSEQ atomic_load() const { auto v = ATOMIC_LOAD(&raw_val_); ObTxSEQ s; s.raw_val_ = v; return s; } @@ -381,6 +383,19 @@ private: }; static_assert(sizeof(ObTxSEQ) == sizeof(int64_t), "ObTxSEQ should sizeof(int64_t)"); +inline OB_DEF_SERIALIZE_SIMPLE(ObTxSEQ) +{ + return serialization::encode_vi64(buf, buf_len, pos, raw_val_); +} +inline OB_DEF_SERIALIZE_SIZE_SIMPLE(ObTxSEQ) +{ + return serialization::encoded_length_vi64(raw_val_); +} +inline OB_DEF_DESERIALIZE_SIMPLE(ObTxSEQ) +{ + return serialization::decode_vi64(buf, data_len, pos, &raw_val_); +} + struct ObLockForReadArg { ObLockForReadArg(memtable::ObMvccAccessCtx &acc_ctx, @@ -801,6 +816,14 @@ public: true, common::ObLatchIds::TRANS_TRACE_RECORDER_LOCK) {} ~ObTransTraceLog() {} void destroy() {} + int64_t to_string(char *buf, const int64_t buf_len) const + { + int64_t ret = 0; + check_lock(); + ret = ObTraceEventRecorder::to_string(buf, buf_len); + check_unlock(); + return ret; + } }; class ObStmtInfo // unreferenced, need remove @@ -1514,21 +1537,31 @@ public: } void destroy() { reset(); } bool is_valid() const - { return undo_from_.is_valid() && undo_to_.is_valid() && undo_from_ > undo_to_; } + { + return (undo_from_.is_valid() + && undo_to_.is_valid() + && undo_from_.get_branch() == undo_to_.get_branch() + && undo_from_ > undo_to_); + } bool is_contain(const ObTxSEQ seq_no) const - { return seq_no > undo_to_ && seq_no <= undo_from_; } + { + const bool can_cmp = undo_to_.get_branch() == 0 + || undo_to_.get_branch() == seq_no.get_branch(); + return can_cmp + && seq_no.get_seq() > undo_to_.get_seq() + && seq_no.get_seq() <= undo_from_.get_seq(); + } bool is_contain(const ObUndoAction &other) const - { return undo_from_ >= other.undo_from_ && undo_to_ <= other.undo_to_; } - - bool is_less_than(const ObTxSEQ seq_no) const - { return seq_no > undo_from_;} - - int merge(const ObUndoAction &other); - + { + const bool can_cmp = undo_to_.get_branch() == 0 + || undo_to_.get_branch() == other.undo_to_.get_branch(); + return can_cmp && + undo_from_.get_seq() >= other.undo_from_.get_seq() + && undo_to_.get_seq() <= other.undo_to_.get_seq(); + } TO_STRING_KV(K_(undo_from), K_(undo_to)); - public: // from > to ObTxSEQ undo_from_; // inclusive @@ -1729,6 +1762,22 @@ class ObTxMDSCache; static const int64_t MAX_TABLET_MODIFY_RECORD_COUNT = 16; // exec info need to be persisted by "trans context table" +template +struct ObIArrayPrintTrait { + const ObIArray &arr_; + ObIArrayPrintTrait(const ObIArray &arr): arr_(arr) {} + DECLARE_TO_STRING + { + int64_t pos = 0; + J_ARRAY_START(); + for(int i =0; i < arr_.count(); i++) { + BUF_PRINTO(arr_.at(i)); + if (i != arr_.count() - 1) { J_COMMA(); } + } + J_ARRAY_END(); + return pos; + } +}; struct ObTxExecInfo { OB_UNIS_VERSION(1); @@ -1739,7 +1788,13 @@ public: incremental_participants_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "INC_PART`")), intermediate_participants_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "INTER_PART`")), redo_lsns_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "REDO_LSNS")), - prepare_log_info_arr_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "PREPARE_INFO")) {} + checksum_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "TX_CHECKSUM")), + checksum_scn_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "TX_CHECKSUM")), + prepare_log_info_arr_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(allocator, "PREPARE_INFO")) + { + checksum_.push_back(0); + checksum_scn_.push_back(share::SCN()); + } public: int generate_mds_buffer_ctx_array(); void mrege_buffer_ctx_array_to_multi_data_source() const; @@ -1755,7 +1810,11 @@ private: const ObTxCommitParts &commit_parts); public: - TO_STRING_KV(K_(state), + DECLARE_TO_STRING { +// const_cast>(checksum_).set_max_print_count(512); +// const_cast>(checksum_scn_).set_max_print_count(512); + int64_t pos = 0; + J_KV(K_(state), K_(upstream), K_(participants), K_(incremental_participants), @@ -1786,7 +1845,11 @@ public: K_(commit_parts), K_(transfer_parts), K_(is_empty_ctx_created_by_transfer), - K_(exec_epoch)); + K_(exec_epoch), + K_(serial_final_scn), + K_(serial_final_seq_no)); + return pos; + } ObTxState state_; share::ObLSID upstream_; share::ObLSArray participants_; @@ -1799,7 +1862,6 @@ public: ObRedoLSNArray redo_lsns_; ObTxBufferNodeArray multi_data_source_; ObTxBufferCtxArray mds_buffer_ctx_array_; - // check common::ObAddr scheduler_; share::SCN prepare_version_; int64_t trans_type_; @@ -1808,8 +1870,8 @@ public: share::SCN max_applying_log_ts_; int64_t max_applying_part_log_no_; // start from 0 on follower and always be INT64_MAX on leader ObTxSEQ max_submitted_seq_no_; // maintains on Leader and transfer to Follower via ActiveInfoLog - uint64_t checksum_; - share::SCN checksum_scn_; + ObSEArray checksum_; + ObSEArray checksum_scn_; palf::LSN max_durable_lsn_; bool data_complete_; bool is_dup_tx_; @@ -1823,6 +1885,13 @@ public: bool is_transfer_blocking_; bool is_empty_ctx_created_by_transfer_; int64_t exec_epoch_; + // if valid, this txCtx is logged by multi parallel thread + // since this scn + share::SCN serial_final_scn_; + // the logic time of serial final log submitted + // used to decide whether a branch level savepoint rollback log + // need set pre-barrier to wait previous redo replayed + ObTxSEQ serial_final_seq_no_; }; static const int64_t USEC_PER_SEC = 1000 * 1000; @@ -1914,6 +1983,65 @@ inline bool is_effective_trans_version(const int64_t trans_version) && INT64_MAX != trans_version; } +template +struct ObIArraySerDeTrait { + ObIArray &arr_; + ObIArraySerDeTrait(ObIArray &arr): arr_(arr) {} + int64_t get_serialize_size() const + { + int64_t size = 0; + size += serialization::encoded_length_vi64(arr_.count()); + for (int64_t i = 0; i < arr_.count(); i ++) { + size += serialization::encoded_length(arr_.at(i)); + } + return size; + } + int serialize(char *buf, const int64_t buf_len, int64_t &pos) const + { + int ret = OB_SUCCESS; + int i = -1; + if (OB_SUCC(serialization::encode_vi64(buf, buf_len, pos, arr_.count()))) { + for (i = 0; i < arr_.count() && OB_SUCC(ret); i ++) { + ret = serialization::encode(buf, buf_len, pos, arr_.at(i)); + } + } + if (OB_FAIL(ret)) { + TRANS_LOG(WARN, "", K(i), K(ret)); + } + return ret; + } + int deserialize(const char *buf, int64_t data_len, int64_t &pos) + { + int ret = OB_SUCCESS; + arr_.reuse(); + bool need_push_back = true; + int64_t count = 0; + if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count))) { + TRANS_LOG(WARN, "decode count fail", K(ret)); + } else if (OB_FAIL(arr_.prepare_allocate(count))) { + if (OB_NOT_SUPPORTED == ret) { + ret = arr_.reserve(count); + } + if (OB_FAIL(ret)) { + TRANS_LOG(WARN, "pre-allocate fail", K(ret), K(count)); + } + } else { need_push_back = false; } + for (int i = 0; i < count && OB_SUCC(ret); i++) { + if (need_push_back) { + T it; + if (OB_FAIL(serialization::decode(buf, data_len, pos, it))) { + TRANS_LOG(WARN, "item decode fail", K(ret), K(i)); + } else if (OB_FAIL(arr_.push_back(it))) { + TRANS_LOG(WARN, "push fail", K(ret), K(i)); + } + } else if (OB_FAIL(serialization::decode(buf, data_len, pos, arr_.at(i)))) { + TRANS_LOG(WARN, "item decode fail", K(ret), K(i)); + } + } + return ret; + } +}; + } // transaction } // oceanbase diff --git a/src/storage/tx/ob_trans_define_v4.cpp b/src/storage/tx/ob_trans_define_v4.cpp index 5f9dc1b703..484d42e9da 100644 --- a/src/storage/tx/ob_trans_define_v4.cpp +++ b/src/storage/tx/ob_trans_define_v4.cpp @@ -287,7 +287,8 @@ OB_SERIALIZE_MEMBER(ObTxDesc, active_scn_, parts_, xid_, - flags_.for_serialize_v_); + flags_.for_serialize_v_, + seq_base_); OB_SERIALIZE_MEMBER(ObTxParam, timeout_us_, lock_timeout_us_, @@ -319,7 +320,8 @@ OB_SERIALIZE_MEMBER(ObTxInfo, active_scn_, parts_, session_id_, - savepoints_); + savepoints_, + seq_base_); OB_SERIALIZE_MEMBER(ObTxStmtInfo, tx_id_, op_sn_, @@ -337,6 +339,7 @@ ObTxDesc::ObTxDesc() cluster_id_(-1), trace_info_(), cluster_version_(0), + seq_base_(0), tx_consistency_type_(ObTxConsistencyType::INVALID), addr_(), tx_id_(), @@ -364,6 +367,7 @@ ObTxDesc::ObTxDesc() finish_ts_(-1), active_scn_(), min_implicit_savepoint_(), + last_branch_id_(0), parts_(), savepoints_(), cflict_txs_(), @@ -481,7 +485,7 @@ void ObTxDesc::reset() cluster_id_ = -1; trace_info_.reset(); cluster_version_ = 0; - + seq_base_ = 0; tx_consistency_type_ = ObTxConsistencyType::INVALID; addr_.reset(); @@ -514,6 +518,7 @@ void ObTxDesc::reset() active_scn_.reset(); min_implicit_savepoint_.reset(); + last_branch_id_ = 0; parts_.reset(); savepoints_.reset(); cflict_txs_.reset(); @@ -1642,7 +1647,6 @@ void TxCtxStateHelper::restore_state() } } -OB_SERIALIZE_MEMBER_SIMPLE(ObTxSEQ, raw_val_); DEF_TO_STRING(ObTxSEQ) { int64_t pos = 0; @@ -1658,21 +1662,18 @@ DEF_TO_STRING(ObTxSEQ) return pos; } -ObTxSEQ ObTxDesc::get_tx_seq(int64_t seq_abs) const +int ObTxDesc::alloc_branch_id(const int64_t count, int16_t &branch_id) { - return ObTxSEQ::mk_v0(seq_abs > 0 ? seq_abs : ObSequence::get_max_seq_no()); -} -ObTxSEQ ObTxDesc::get_and_inc_tx_seq(int16_t branch, int N) const -{ - UNUSED(branch); - int64_t seq = ObSequence::get_and_inc_max_seq_no(N); - return ObTxSEQ::mk_v0(seq); -} -ObTxSEQ ObTxDesc::inc_and_get_tx_seq(int16_t branch) const -{ - UNUSED(branch); - int64_t seq = ObSequence::inc_and_get_max_seq_no(); - return ObTxSEQ::mk_v0(seq); + int ret = OB_SUCCESS; + ObSpinLockGuard guard(lock_); + if (count > MAX_BRANCH_ID_VALUE - last_branch_id_) { + ret = OB_ERR_OUT_OF_UPPER_BOUND; + TRANS_LOG(WARN, "can not alloc branch_id", KR(ret), K(count), KPC(this)); + } else { + branch_id = last_branch_id_ + 1; + last_branch_id_ += count; + } + return ret; } void ObTxDesc::mark_part_abort(const ObTransID tx_id, const int abort_cause) { diff --git a/src/storage/tx/ob_trans_define_v4.h b/src/storage/tx/ob_trans_define_v4.h index 68f7eadfcd..4534480583 100644 --- a/src/storage/tx/ob_trans_define_v4.h +++ b/src/storage/tx/ob_trans_define_v4.h @@ -430,7 +430,7 @@ protected: int64_t cluster_id_; ObTraceInfo trace_info_; uint64_t cluster_version_; // compatible handle when upgrade - + int64_t seq_base_; // tx_seq's base value, use to calculate absolute value of tx_seq ObTxConsistencyType tx_consistency_type_; // transaction level consistency_type : strong or bounded read common::ObAddr addr_; // where we site @@ -531,6 +531,7 @@ protected: ObTxSEQ active_scn_; // logical time of ACTIVE | IMPLICIT_ACTIVE ObTxSEQ min_implicit_savepoint_; // mininum of implicit savepoints + int16_t last_branch_id_; // branch_id allocator, reset when stmt start ObTxPartList parts_; // participant list ObTxSavePointList savepoints_; // savepoints established // cflict_txs_ is used to store conflict trans id when try acquire row lock failed(meet lock conflict) @@ -582,6 +583,7 @@ private: DLink *prev_; } alloc_link_; #endif + static constexpr int16_t MAX_BRANCH_ID_VALUE = ~(1 << 15) & 0xFFFF; // 15bits private: /* these routine should be called by txn-service only to avoid corrupted state */ void reset(); @@ -643,6 +645,7 @@ public: KP_(commit_cb), K_(cluster_id), K_(cluster_version), + K_(seq_base), K_(flags_.SHADOW), K_(flags_.INTERRUPTED), K_(flags_.BLOCK), @@ -653,7 +656,11 @@ public: K_(commit_expire_ts), K(commit_task_.is_registered()), K_(ref)); - + bool support_branch() const { return seq_base_ > 0; } + // used by SQL alloc branch_id refer the min branch_id allowed + // because branch_id bellow this is reserved for internal use + int branch_id_offset() const { return MAX_CALLBACK_LIST_COUNT; } + int alloc_branch_id(const int64_t count, int16_t &branch_id); int fetch_conflict_txs(ObIArray &array); void reset_conflict_txs() { ObSpinLockGuard guard(lock_); cflict_txs_.reset(); } @@ -786,6 +793,7 @@ LST_DO(DEF_FREE_ROUTE_DECODE, (;), static, dynamic, parts, extra); ObTxSEQ get_and_inc_tx_seq(int16_t branch, int N) const; ObTxSEQ inc_and_get_tx_seq(int16_t branch) const; ObTxSEQ get_tx_seq(int64_t seq_abs = 0) const; + ObTxSEQ get_min_tx_seq() const; }; // Is used to store and travserse all TxScheduler's Stat information; @@ -886,6 +894,7 @@ protected: uint64_t tenant_id_; int64_t cluster_id_; uint64_t cluster_version_; + int64_t seq_base_; common::ObAddr addr_; ObTransID tx_id_; ObTxIsolationLevel isolation_; @@ -903,6 +912,7 @@ protected: uint32_t session_id_ = 0; ObTxSavePointList savepoints_; public: + ObTxInfo(): seq_base_(0) {} TO_STRING_KV(K_(tenant_id), K_(session_id), K_(tx_id), @@ -915,6 +925,7 @@ public: K_(active_ts), K_(timeout_us), K_(expire_ts), + K_(seq_base), K_(parts), K_(cluster_id), K_(cluster_version), @@ -1006,6 +1017,48 @@ typedef lib::ObLockGuardWithTimeout ObSpinLockGuardWithTimeout; } \ } while (0) +inline ObTxSEQ ObTxDesc::get_tx_seq(int64_t seq_abs) const +{ + int64_t seq = seq_abs > 0 ? seq_abs : ObSequence::get_max_seq_no(); + if (OB_LIKELY(support_branch())) { + if (seq < seq_base_) { + TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "seq_abs is less seq_base_", K(seq_abs), K(tx_id_), K(seq_base_)); + return ObTxSEQ::INVL(); + } + return ObTxSEQ(seq - seq_base_, 0); + } else { + return ObTxSEQ::mk_v0(seq); + } +} + +inline ObTxSEQ ObTxDesc::get_min_tx_seq() const +{ + if (OB_LIKELY(support_branch())) { + return ObTxSEQ(1, 0); + } else { + return ObTxSEQ::mk_v0(1); + } +} + +inline ObTxSEQ ObTxDesc::get_and_inc_tx_seq(int16_t branch, int N) const +{ + int64_t seq = ObSequence::get_and_inc_max_seq_no(N); + if (OB_LIKELY(support_branch())) { + return ObTxSEQ(seq - seq_base_, branch); + } else { + return ObTxSEQ::mk_v0(seq); + } +} + +inline ObTxSEQ ObTxDesc::inc_and_get_tx_seq(int16_t branch) const +{ + int64_t seq = ObSequence::inc_and_get_max_seq_no(); + if (OB_LIKELY(support_branch())) { + return ObTxSEQ(seq - seq_base_, branch); + } else { + return ObTxSEQ::mk_v0(seq); + } +} } // transaction } // oceanbase diff --git a/src/storage/tx/ob_trans_functor.h b/src/storage/tx/ob_trans_functor.h index 5deb7d1d78..34ea8d0a67 100644 --- a/src/storage/tx/ob_trans_functor.h +++ b/src/storage/tx/ob_trans_functor.h @@ -955,39 +955,50 @@ public: tx_ctx->print_trace_log(); } if (OB_SUCC(ret)) { + int tmp_ret = OB_SUCCESS; share::ObLSArray participants_arr; ObTxData *tx_data = NULL; + int busy_cbs_cnt = -1; tx_ctx->ctx_tx_data_.get_tx_data_ptr(tx_data); - if (OB_FAIL(tx_ctx->get_2pc_participants_copy(participants_arr))) { - TRANS_LOG_RET(WARN, ret, "ObTxStat get participants copy error", K(ret)); - } else if (OB_FAIL(tx_stat.init(tx_ctx->addr_, - tx_id, - tx_ctx->tenant_id_, - has_decided, - tx_ctx->ls_id_, - participants_arr, - tx_ctx->ctx_create_time_, - tx_ctx->trans_expired_time_, - tx_ctx->ref_, - tx_ctx->last_op_sn_, - tx_ctx->pending_write_, - (int64_t)tx_ctx->exec_info_.state_, - tx_ctx->exec_info_.trans_type_, - tx_ctx->part_trans_action_, - tx_ctx, - tx_ctx->get_pending_log_size(), - tx_ctx->get_flushed_log_size(), - tx_ctx->role_state_, - tx_ctx->session_id_, - tx_ctx->exec_info_.scheduler_, - tx_ctx->is_exiting_, - tx_ctx->exec_info_.xid_, - tx_ctx->exec_info_.upstream_, - tx_ctx->last_request_ts_, - OB_NOT_NULL(tx_data) ? tx_data->start_scn_.atomic_load() : SCN::invalid_scn(), - OB_NOT_NULL(tx_data) ? tx_data->end_scn_.atomic_load() : SCN::invalid_scn(), - tx_ctx->get_rec_log_ts_(), - tx_ctx->sub_state_.is_transfer_blocking()))) { + if (OB_TMP_FAIL(tx_ctx->get_stat_for_virtual_table(participants_arr, busy_cbs_cnt))) { + TRANS_LOG_RET(WARN, tmp_ret, "ObTxStat get participants copy error", K(tmp_ret)); + // push an invalid ls id to hint the failure + participants_arr.push_back(share::ObLSID()); + } + if (OB_TMP_FAIL(tx_ctx->mt_ctx_.get_callback_list_stat(tx_stat.callback_list_stats_))) { + TRANS_LOG_RET(WARN, tmp_ret, "ObTxStat get callback lists stat error", K(tmp_ret)); + } + if (OB_FAIL(tx_stat.init(tx_ctx->addr_, + tx_id, + tx_ctx->tenant_id_, + has_decided, + tx_ctx->ls_id_, + participants_arr, + tx_ctx->ctx_create_time_, + tx_ctx->trans_expired_time_, + tx_ctx->ref_, + tx_ctx->last_op_sn_, + tx_ctx->pending_write_, + (int64_t)tx_ctx->exec_info_.state_, + tx_ctx->exec_info_.trans_type_, + tx_ctx->part_trans_action_, + tx_ctx, + tx_ctx->get_pending_log_size(), + tx_ctx->get_flushed_log_size(), + tx_ctx->role_state_, + tx_ctx->session_id_, + tx_ctx->exec_info_.scheduler_, + tx_ctx->is_exiting_, + tx_ctx->exec_info_.xid_, + tx_ctx->exec_info_.upstream_, + tx_ctx->last_request_ts_, + OB_NOT_NULL(tx_data) ? tx_data->start_scn_.atomic_load() : SCN::invalid_scn(), + OB_NOT_NULL(tx_data) ? tx_data->end_scn_.atomic_load() : SCN::invalid_scn(), + tx_ctx->get_rec_log_ts_(), + tx_ctx->sub_state_.is_transfer_blocking(), + busy_cbs_cnt, + (int)tx_ctx->replay_completeness_.complete_, + tx_ctx->exec_info_.serial_final_scn_))) { TRANS_LOG_RET(WARN, ret, "ObTxStat init error", K(ret), KPC(tx_ctx)); } else if (OB_FAIL(tx_stat_iter_.push(tx_stat))) { TRANS_LOG_RET(WARN, ret, "ObTxStatIterator push trans stat error", K(ret)); @@ -1302,7 +1313,7 @@ public: ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", K(ret), K(tx_id), "ctx", OB_P(tx_ctx)); } else if (ObTxSubmitLogFunctor::SUBMIT_REDO_LOG == action_) { - if (OB_FAIL(tx_ctx->submit_redo_log(true))) { + if (OB_FAIL(tx_ctx->submit_redo_log_for_freeze())) { TRANS_LOG(WARN, "failed to submit redo log", K(ret), K(tx_id)); } } else if (ObTxSubmitLogFunctor::SUBMIT_NEXT_LOG == action_) { diff --git a/src/storage/tx/ob_trans_part_ctx.cpp b/src/storage/tx/ob_trans_part_ctx.cpp index 7a62596229..173c7ebe3a 100644 --- a/src/storage/tx/ob_trans_part_ctx.cpp +++ b/src/storage/tx/ob_trans_part_ctx.cpp @@ -20,10 +20,12 @@ #include "common/storage/ob_sequence.h" #include "lib/profile/ob_perf_event.h" #include "lib/utility/serialization.h" +#include "lib/container/ob_array_helper.h" #include "ob_trans_ctx_mgr.h" #include "ob_ts_mgr.h" #include "ob_tx_log.h" #include "ob_tx_msg.h" +#include "ob_tx_redo_submitter.h" #include "lib/worker.h" #include "share/rc/ob_context.h" #include "storage/memtable/ob_lock_wait_mgr.h" @@ -90,7 +92,6 @@ int ObPartTransCtx::init(const uint64_t tenant_id, int ret = OB_SUCCESS; CtxLockGuard guard(lock_); - // default init : just reset immediately default_init_(); @@ -99,7 +100,7 @@ int ObPartTransCtx::init(const uint64_t tenant_id, TRANS_LOG(WARN, "ObPartTransCtx inited twice"); ret = OB_INIT_TWICE; } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id)) || OB_UNLIKELY(!trans_id.is_valid()) - || OB_UNLIKELY(trans_expired_time <= 0) || OB_UNLIKELY(cluster_version <= 0) + || OB_UNLIKELY(trans_expired_time <= 0) || OB_UNLIKELY(cluster_version < 0) || OB_ISNULL(trans_service) || OB_UNLIKELY(!ls_id.is_valid())) { TRANS_LOG(WARN, "invalid argument", K(tenant_id), K(trans_id), K(epoch), K(cluster_version), KP(trans_service), K(ls_id)); @@ -134,7 +135,8 @@ int ObPartTransCtx::init(const uint64_t tenant_id, trans_id_ = trans_id; trans_expired_time_ = trans_expired_time; ctx_create_time_ = ObClockGenerator::getClock(); - cluster_version_ = cluster_version; + cluster_version_accurate_ = cluster_version > 0; + cluster_version_ = cluster_version ?: LAST_BARRIER_DATA_VERSION; part_trans_action_ = ObPartTransAction::INIT; trans_2pc_timeout_ = ObServerConfig::get_instance().trx_2pc_retry_interval; last_request_ts_ = ctx_create_time_; @@ -241,7 +243,6 @@ void ObPartTransCtx::destroy() K(busy_cbs_.get_size())); } - if (exec_info_.is_dup_tx_ && OB_NOT_NULL(ls_tx_ctx_mgr_)) { if (OB_FAIL(ls_tx_ctx_mgr_->get_ls_log_adapter()->remove_commiting_dup_trx(trans_id_))) { TRANS_LOG(WARN, "remove committing dup table trx failed", K(ret), KPC(this)); @@ -355,7 +356,7 @@ void ObPartTransCtx::default_init_() retain_ctx_func_ptr_ = nullptr; start_replay_ts_.reset(); start_recover_ts_.reset(); - is_incomplete_replay_ctx_ = false; + replay_completeness_.reset(); is_submitting_redo_log_for_freeze_ = false; start_working_log_ts_ = SCN::min_scn(); max_2pc_commit_scn_.reset(); @@ -391,11 +392,10 @@ int ObPartTransCtx::init_log_cbs_(const ObLSID &ls_id, const ObTransID &tx_id) return ret; } -int ObPartTransCtx::extend_log_cbs_() +int ObPartTransCtx::extend_log_cbs_(ObTxLogCb *&cb) { int ret = OB_SUCCESS; void *ptr = NULL; - ObTxLogCb *cb = NULL; if (busy_cbs_.get_size() >= OB_TX_MAX_LOG_CBS + 1) { ret = OB_TX_NOLOGCB; } else if (ATOMIC_LOAD(&is_submitting_redo_log_for_freeze_) == false && @@ -410,15 +410,10 @@ int ObPartTransCtx::extend_log_cbs_() } else if (OB_FAIL(cb->init(ls_id_, trans_id_, this, true))) { TRANS_LOG(WARN, "log callback init failed", K(ret)); cb->~ObTxLogCb(); - } else if (!free_cbs_.add_last(cb)) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "add to free list failed", KR(ret)); - cb->~ObTxLogCb(); - } else { - // do nothing } if (OB_FAIL(ret)) { reserve_allocator_.free(ptr); + cb = NULL; } } return ret; @@ -474,6 +469,7 @@ int ObPartTransCtx::start_trans() TRANS_LOG(WARN, "transaction is timeout", K(ret), K_(trans_expired_time), KPC(this)); } else { part_trans_action_ = ObPartTransAction::START; + replay_completeness_.set(true); if (left_time > 0 && left_time < default_timeout_us) { (void)unregister_timeout_task_(); if (OB_FAIL(register_timeout_task_(left_time))) { @@ -802,7 +798,12 @@ int ObPartTransCtx::commit(const ObTxCommitParts &parts, int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; CtxLockGuard guard(lock_); - + if (tenant_id_ % 2 == 0) { + TRANS_LOG(DEBUG, "commit transaction now!", "trans_id", get_trans_id()); + } + if (is_parallel_logging()) { + TRANS_LOG(INFO, "commit transaction now!", KPC(this)); + } if (IS_NOT_INIT) { TRANS_LOG(WARN, "ObPartTransCtx not inited"); ret = OB_NOT_INIT; @@ -1002,19 +1003,6 @@ int ObPartTransCtx::iterate_tx_obj_lock_op(ObLockOpIterator &iter) const return ret; } -bool ObPartTransCtx::need_update_schema_version(const int64_t log_id, const SCN) -{ - // const int64_t restore_snapshot_version = ls_tx_ctx_mgr_->get_restore_snapshot_version(); - // const int64_t last_restore_log_id = ls_tx_ctx_mgr_->get_last_restore_log_id(); - bool need_update = true; - // if (restore_snapshot_version > 0 - // && (last_restore_log_id == OB_INVALID_ID || log_id <= last_restore_log_id) - // && (log_ts > restore_snapshot_version)) { - // need_update = false; - //} - return need_update; -} - int ObPartTransCtx::trans_replay_abort_(const SCN &final_log_ts) { int ret = OB_SUCCESS; @@ -1040,7 +1028,7 @@ int ObPartTransCtx::trans_replay_abort_(const SCN &final_log_ts) int ObPartTransCtx::trans_replay_commit_(const SCN &commit_version, const SCN &final_log_ts, const uint64_t log_cluster_version, - const int64_t checksum) + const uint64_t checksum) { ObTimeGuard tg("trans_replay_commit", 50 * 1000); int ret = OB_SUCCESS; @@ -1131,7 +1119,11 @@ bool ObPartTransCtx::is_force_abort_logging_() const bool ObPartTransCtx::has_persisted_log_() const { - return exec_info_.max_applying_log_ts_.is_valid(); + return exec_info_.max_applying_log_ts_.is_valid() || + // for ctx created by parallel replay, and no serial log replayed + // the max_applying_log_ts_ has not been set, in this case use + // replay_completeness to distinguish with situations on leader + (is_follower_() && replay_completeness_.is_unknown()); } int ObPartTransCtx::gts_callback_interrupted(const int errcode) @@ -1530,7 +1522,7 @@ int ObPartTransCtx::recover_tx_ctx_table_info(ObTxCtxTableInfo &ctx_info) // TRANS_LOG(WARN, "unexpected null ptr", K(*this)); } else if (OB_FAIL(mt_ctx_.recover_from_table_lock_durable_info(ctx_info.table_lock_info_))) { TRANS_LOG(ERROR, "recover_from_table_lock_durable_info failed", K(ret)); - } else if (OB_FAIL(ctx_tx_data_.recover_tx_data(ctx_info.tx_data_guard_))) { + } else if (OB_FAIL(ctx_tx_data_.recover_tx_data(ctx_info.tx_data_guard_.tx_data()))) { TRANS_LOG(WARN, "recover tx data failed", K(ret), K(ctx_tx_data_)); } else if (OB_FAIL(exec_info_.assign(ctx_info.exec_info_))) { TRANS_LOG(WARN, "exec_info assign error", K(ret), K(ctx_info)); @@ -1555,13 +1547,14 @@ int ObPartTransCtx::recover_tx_ctx_table_info(ObTxCtxTableInfo &ctx_info) // do nothing } else if (OB_FAIL(deep_copy_mds_array_(ctx_info.exec_info_.multi_data_source_, _unused_))) { TRANS_LOG(WARN, "deep copy ctx_info mds_array failed", K(ret)); - } else if (FALSE_IT(mt_ctx_.update_checksum(exec_info_.checksum_, - exec_info_.checksum_scn_))) { - TRANS_LOG(ERROR, "recover checksum failed", K(ret), KPC(this), K(ctx_info)); + } else if (OB_FAIL(mt_ctx_.update_checksum(exec_info_.checksum_, + exec_info_.checksum_scn_))) { + TRANS_LOG(WARN, "recover checksum failed", K(ret), KPC(this), K(ctx_info)); } else if (!is_local_tx_() && OB_FAIL(ObTxCycleTwoPhaseCommitter::recover_from_tx_table())) { TRANS_LOG(ERROR, "recover_from_tx_table failed", K(ret), KPC(this)); } else { is_ctx_table_merged_ = true; + replay_completeness_.set(true); } if (OB_FAIL(ret)) { @@ -1620,6 +1613,9 @@ int ObPartTransCtx::recover_tx_ctx_table_info(ObTxCtxTableInfo &ctx_info) } if (OB_SUCC(ret)) { + if (exec_info_.serial_final_scn_.is_valid()) { + recovery_parallel_logging_(); + } if ((ObTxState::COMMIT == exec_info_.state_ && exec_info_.max_applying_log_ts_ == exec_info_.max_applied_log_ts_) || ObTxState::CLEAR == exec_info_.state_) { @@ -1664,25 +1660,25 @@ int ObPartTransCtx::serialize_tx_ctx_to_buffer(ObTxLocalBuffer &buffer, int64_t ret = OB_TRANS_CTX_NOT_EXIST; TRANS_LOG(INFO, "tx ctx has no persisted log", K(ret), KPC(this)); // 3. Tx ctx has no persisted log, so donot need persisting - } else if (is_incomplete_replay_ctx_) { + } else if (!replay_completeness_.is_complete()) { // NB: we need refresh rec log ts for incomplete replay ctx if (OB_FAIL(refresh_rec_log_ts_())) { TRANS_LOG(WARN, "refresh rec log ts failed", K(ret), KPC(this)); } else { ret = OB_TRANS_CTX_NOT_EXIST; - TRANS_LOG(INFO, "tx ctx is in complete replay ctx", K(ret), KPC(this)); + TRANS_LOG(INFO, "tx ctx is an incomplete replay ctx", K(ret), KPC(this)); } - // 3. Fetch the current state of the tx ctx table + // 4. Fetch the current state of the tx ctx table } else if (OB_FAIL(get_tx_ctx_table_info_(ctx_info))) { TRANS_LOG(WARN, "get tx ctx table info failed", K(ret), K(*this)); } else if (OB_UNLIKELY(!ctx_info.is_valid())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "tx ctx info invalid", K(ret), K(ctx_info)); - // 4. Refresh the rec_log_ts for the next checkpoint + // 5. Refresh the rec_log_ts for the next checkpoint } else if (OB_FAIL(refresh_rec_log_ts_())) { TRANS_LOG(WARN, "refresh rec log ts failed", K(ret), K(*this)); } else { - // 5. Do serialize + // 6. Do serialize int64_t pos = 0; serialize_size = ctx_info.get_serialize_size(); if (OB_FAIL(buffer.reserve(serialize_size))) { @@ -1754,7 +1750,7 @@ int ObPartTransCtx::remove_callback_for_uncommited_txn( } else if (OB_ISNULL(memtable_set)) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "memtable is NULL", K(memtable_set)); - } else if (OB_FAIL(mt_ctx_.remove_callback_for_uncommited_txn(memtable_set, exec_info_.max_applied_log_ts_))) { + } else if (OB_FAIL(mt_ctx_.remove_callback_for_uncommited_txn(memtable_set))) { TRANS_LOG(WARN, "fail to remove callback for uncommitted txn", K(ret), K(mt_ctx_), K(memtable_set), K(exec_info_.max_applied_log_ts_)); } @@ -1762,89 +1758,168 @@ int ObPartTransCtx::remove_callback_for_uncommited_txn( return ret; } -// this function is only called by instant logging and freezing, -// they both view ret code OB_BLOCK_FROZEN as success. -int ObPartTransCtx::submit_redo_log(const bool is_freeze) +// the semantic of submit redo for freeze is +// should flush all redos which can be flushed +// otherwise, need return some error to caller +// to indicate need retry +int ObPartTransCtx::submit_redo_log_for_freeze() { int ret = OB_SUCCESS; - ObTimeGuard tg("submit_redo_log", 100000); - bool try_submit = false; - - if (is_freeze) { - bool need_submit = !is_logging_blocked(); - if (need_submit) { - do { - // spin lock - CtxLockGuard guard(lock_); - tg.click(); - ret = submit_redo_log_for_freeze_(try_submit); - tg.click(); - if (try_submit) { - REC_TRANS_TRACE_EXT2(tlog_, submit_instant_log, OB_Y(ret), OB_ID(arg2), is_freeze, - OB_ID(used), tg.get_diff(), OB_ID(ref), get_ref()); - } - } while (OB_EAGAIN == ret); + TRANS_LOG(TRACE, "", K_(trans_id), K_(ls_id)); + ObTimeGuard tg("submit_redo_for_freeze_log", 100000); + bool submitted = false; + bool need_submit = !is_logging_blocked(); + if (need_submit) { + CtxLockGuard guard(lock_); + tg.click(); + ret = submit_redo_log_for_freeze_(submitted); + tg.click(); + if (submitted) { + REC_TRANS_TRACE_EXT2(tlog_, submit_log_for_freeze, OB_Y(ret), + OB_ID(used), tg.get_diff(), OB_ID(ref), get_ref()); } - } else if (!mt_ctx_.pending_log_size_too_large()) { - } else if (OB_FAIL(lock_.try_lock())) { - // try lock - if (OB_EAGAIN == ret) { + // TODO: mark frozen memtable for fast check need submit redo + // if (OB_BLOCK_FROZEN != ret) { + // clear_block_frozen_memtable(); + // } + if (OB_TRANS_HAS_DECIDED == ret || OB_BLOCK_FROZEN == ret) { ret = OB_SUCCESS; - } else { - TRANS_LOG(ERROR, "try lock error, unexpected error", K(ret), K(*this)); } - } else { - bool need_lock = false; - do { - CtxLockGuard guard(lock_, need_lock); - tg.click(); - ret = check_and_submit_redo_log_(try_submit); - tg.click(); - if (try_submit) { - REC_TRANS_TRACE_EXT2(tlog_, submit_instant_log, OB_Y(ret), OB_ID(arg2), is_freeze, - OB_ID(used), tg.get_diff(), - OB_ID(ref), get_ref()); - } - need_lock = true; - } while (OB_EAGAIN == ret); - } - if (OB_BLOCK_FROZEN == ret) { - ret = OB_SUCCESS; - } else { - clear_block_frozen_memtable(); } return ret; } -int ObPartTransCtx::check_and_submit_redo_log_(bool &try_submit) +int ObPartTransCtx::submit_redo_after_write(const bool force, const ObTxSEQ &write_seq_no) +{ + int ret = OB_SUCCESS; + TRANS_LOG(TRACE, "", K(force), K(write_seq_no), K_(trans_id), K_(ls_id), + K(mt_ctx_.get_pending_log_size())); + ObTimeGuard tg("submit_redo_for_after_write", 100000); + if (force || mt_ctx_.pending_log_size_too_large(write_seq_no)) { + bool parallel_logging = false; +#define LOAD_PARALLEL_LOGGING parallel_logging = exec_info_.serial_final_scn_.atomic_load().is_valid() + LOAD_PARALLEL_LOGGING; + if (!parallel_logging) { + if (OB_SUCCESS == lock_.try_lock()) { + CtxLockGuard guard(lock_, false); + // double check parallel_logging is on + LOAD_PARALLEL_LOGGING; + if (!parallel_logging) { + ret = serial_submit_redo_after_write_(); + } + } + if (OB_BUF_NOT_ENOUGH == ret) { + // bufer full, try fill after switch to parallel logging + LOAD_PARALLEL_LOGGING; + } + } +#undef LOAD_PARALLEL_LOGGING + tg.click("serial_log"); + if (parallel_logging) { + if (OB_SUCC(lock_.try_rdlock_flush_redo())) { + if (OB_SUCC(check_can_submit_redo_()) && !is_committing_()) { + ObTxRedoSubmitter submitter(*this, mt_ctx_); + if (OB_FAIL(submitter.parallel_submit(write_seq_no))) { + if (OB_ITER_END == ret || OB_EAGAIN == ret) { + ret = OB_SUCCESS; + } + } + } + lock_.unlock_flush_redo(); + } + } + } + // TODO: mark frozen memtable for fast check need submit redo + // if (OB_BLOCK_FROZEN != ret) { + // clear_block_frozen_memtable(); + // } + if (OB_TRANS_HAS_DECIDED == ret || OB_BLOCK_FROZEN == ret) { + ret = OB_SUCCESS; + } + return ret; +} + +int ObPartTransCtx::serial_submit_redo_after_write_() +{ + int ret = OB_SUCCESS; + if (OB_SUCC(check_can_submit_redo_())) { + int64_t before_submit_pending_size = mt_ctx_.get_pending_log_size(); + bool should_switch = should_switch_to_parallel_logging_(); + ObTxRedoSubmitter submitter(*this, mt_ctx_); + ret = submitter.submit(false /*flush all log*/, should_switch, false /*display blocked info*/); + if (should_switch && submitter.get_submitted_cnt() > 0) { + const share::SCN serial_final_scn = submitter.get_submitted_scn(); + switch_to_parallel_logging_(serial_final_scn); + TRANS_LOG(INFO, "**switch to parallel logging**", + K_(ls_id), K_(trans_id), + K(serial_final_scn), + "serial_final_seq_no", exec_info_.serial_final_seq_no_, + K(before_submit_pending_size), + "curr_pending_size", mt_ctx_.get_pending_log_size()); + } + } + return ret; +} + +bool ObPartTransCtx::should_switch_to_parallel_logging_() +{ + const int64_t switch_size = GCONF._parallel_redo_logging_trigger; + return GCONF._enable_parallel_redo_logging + && is_support_parallel_replay_() + && pending_write_ > 1 + && mt_ctx_.get_pending_log_size() > switch_size; +} + +int ObPartTransCtx::check_can_submit_redo_() { int ret = OB_SUCCESS; bool is_tx_committing = ObTxState::INIT != get_downstream_state(); bool final_log_submitting = final_log_cb_.is_valid(); - if (!is_tx_committing && !final_log_submitting && !is_force_abort_logging_()) { - (void)mt_ctx_.merge_multi_callback_lists_for_immediate_logging(); - ret = submit_log_impl_(ObTxLogType::TX_REDO_LOG); - try_submit = true; + if (is_tx_committing + ||final_log_submitting + || is_force_abort_logging_()) { + ret = OB_TRANS_HAS_DECIDED; } - return ret; } -int ObPartTransCtx::submit_redo_log_for_freeze_(bool &try_submit) +// Concurrency safe annotation: +// init log_block_ is an local operation +// prepare_log_cb_ is protected by `log_cb_lock_` +int ObPartTransCtx::prepare_for_submit_redo(ObTxLogCb *&log_cb, + ObTxLogBlock &log_block, + const bool serial_final) { int ret = OB_SUCCESS; - - ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, true); - if (OB_FAIL(check_and_submit_redo_log_(try_submit))) { - TRANS_LOG(WARN, "fail to submit redo log for freeze", K(ret)); + if (!log_block.is_inited() && OB_FAIL(init_log_block_(log_block, ObTxAdaptiveLogBuf::NORMAL_LOG_BUF_SIZE, serial_final))) { + TRANS_LOG(WARN, "init log block fail", K(ret)); + } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb)) && OB_TX_NOLOGCB != ret) { + TRANS_LOG(WARN, "prepare log_cb fail", K(ret)); } - if (try_submit && (OB_SUCC(ret) || OB_BLOCK_FROZEN == ret)) { + return ret; +} + +int ObPartTransCtx::submit_redo_log_for_freeze_(bool &submitted) +{ + int ret = OB_SUCCESS; + ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, true); + if (OB_SUCC(check_can_submit_redo_())) { + ObTxRedoSubmitter submitter(*this, mt_ctx_); + if (OB_FAIL(submitter.submit(true/*flush all*/, false/*final serial*/, true /*display blocked info*/))) { + if (OB_BLOCK_FROZEN != ret) { + TRANS_LOG(WARN, "fail to submit redo log for freeze", K(ret)); + // for some error, txn will be aborted immediately + handle_submit_log_err_(ObTxLogType::TX_REDO_LOG, ret); + } + } + submitted = submitter.get_submitted_cnt() > 0; + } + if (OB_SUCC(ret) || OB_BLOCK_FROZEN == ret) { ret = submit_log_impl_(ObTxLogType::TX_MULTI_DATA_SOURCE_LOG); } ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, false); - return ret; } @@ -2015,28 +2090,40 @@ int ObPartTransCtx::on_dist_end_(const bool commit) return ret; } - int ObPartTransCtx::on_success(ObTxLogCb *log_cb) { int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; const int64_t cur_ts = ObTimeUtility::current_time(); - ObTransStatistic::get_instance().add_clog_sync_time(tenant_id_, cur_ts - log_cb->get_submit_ts()); + const int64_t log_sync_used_time = cur_ts - log_cb->get_submit_ts(); + ObTransStatistic::get_instance().add_clog_sync_time(tenant_id_, log_sync_used_time); ObTransStatistic::get_instance().add_clog_sync_count(tenant_id_, 1); + bool skip_fast_commit = false; + bool try_submit_next_log = false; { - CtxLockGuard guard(lock_); - const SCN log_ts = log_cb->get_log_ts(); - const ObTxLogType last_log_type = log_cb->get_last_log_type(); - if (is_exiting_) { + // allow fill redo concurrently with log callback + CtxLockGuard guard(lock_, is_committing_() ? CtxLockGuard::MODE::ALL : CtxLockGuard::MODE::CTX); + const int64_t ctx_lock_wait_time = guard.get_lock_acquire_used_time(); + if (log_sync_used_time + ctx_lock_wait_time >= ObServerConfig::get_instance().clog_sync_time_warn_threshold) { + TRANS_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "transaction log sync use too much time", KPC(log_cb), + K(log_sync_used_time), K(ctx_lock_wait_time)); + } + if (log_cb->get_cb_arg_array().count() == 0) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "cb arg array is empty", K(ret), KPC(this)); + print_trace_log_(); + usleep(5000); + ob_abort(); + } + if (log_cb->is_callbacked()) { +#ifndef NDEBUG + TRANS_LOG(INFO, "cb has been callbacked", KPC(log_cb)); +#endif + } else if (is_exiting_) { // maybe because commit log callbacked before redo log, and ctx is already released // if there is tx data, it would be released when reset_log_cbs_ -#ifndef NDEBUG - TRANS_LOG(INFO, "tx ctx exiting", K(*this)); -#endif - } else if (log_cb->is_callbacked()) { -#ifndef NDEBUG - TRANS_LOG(INFO, "cb has been callbacked", K(*this), K(*log_cb)); -#endif + TRANS_LOG(ERROR, "this callback was missed when tx ctx exiting", KPC(log_cb)); + ob_abort(); } else { // save the first error code int save_ret = OB_SUCCESS; @@ -2066,6 +2153,7 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) } } if (cur_cb != log_cb) { + ob_abort(); ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "unexpected log callback", K(ret), K(*this), K(*cur_cb), K(*log_cb)); } else { @@ -2073,20 +2161,32 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) ret = save_ret; } } - // ignore ret - return_log_cb_(log_cb); + if (sub_state_.is_state_log_submitted() || log_cb->get_callbacks().count() == 0) { + skip_fast_commit = true; + } if (need_record_log_()) { // ignore error if (OB_SUCCESS != (tmp_ret = submit_record_log_())) { TRANS_LOG(WARN, "failed to submit record log", K(tmp_ret), K(*this)); } } - if (!ObTxLogTypeChecker::is_state_log(last_log_type)) { - try_submit_next_log_(); - } - // REC_TRANS_TRACE_EXT(tlog_, on_succ_cb, OB_ID(ret), ret, - // OB_ID(t), log_ts, - // OB_ID(ref), get_ref()); + try_submit_next_log = !ObTxLogTypeChecker::is_state_log(log_cb->get_last_log_type()) && is_committing_(); + busy_cbs_.remove(log_cb); + } + // let fast commit out of ctx's lock, because it is time consuming in calculating checksum + if (!skip_fast_commit) { + // acquire REDO_FLUSH_READ LOCK, which allow other thread flush redo + // but disable other manage operation on ctx + // FIXME: acquire CTX's READ lock maybe better + CtxLockGuard guard(lock_, CtxLockGuard::MODE::REDO_FLUSH_R); + mt_ctx_.remove_callbacks_for_fast_commit(log_cb->get_callbacks()); + } + return_log_cb_(log_cb); + // try submit log if txn is in commit phase + if (try_submit_next_log) { + // in commiting, acquire CTX lock is enough, because redo flushing must finished + CtxLockGuard guard(lock_, CtxLockGuard::MODE::CTX); + try_submit_next_log_(false); } if (OB_SUCCESS != (tmp_ret = ls_tx_ctx_mgr_->revert_tx_ctx_without_lock(this))) { TRANS_LOG(ERROR, "release ctx ref failed", KR(tmp_ret)); @@ -2335,8 +2435,6 @@ int ObPartTransCtx::common_on_success_(ObTxLogCb *log_cb) if (OB_SUCC(ret)) { if (OB_FAIL(mt_ctx_.sync_log_succ(log_ts, log_cb->get_callbacks()))) { TRANS_LOG(ERROR, "mt ctx sync log failed", KR(ret), K(*log_cb), K(*this)); - } else if (OB_SUCCESS != (tmp_ret = mt_ctx_.remove_callbacks_for_fast_commit())) { - TRANS_LOG(WARN, "cleanout callbacks for fast commit", K(ret), K(*this)); } } return ret; @@ -2364,7 +2462,8 @@ int ObPartTransCtx::try_submit_next_log_(const bool for_freeze) ObTxLogType log_type = ObTxLogType::UNKNOWN; if (ObPartTransAction::COMMIT == part_trans_action_ && !is_in_2pc_() - && !need_force_abort_()) { + && !need_force_abort_() + && !is_2pc_blocking()) { if (is_follower_()) { ret = OB_NOT_MASTER; } else { @@ -2412,85 +2511,107 @@ int ObPartTransCtx::fix_redo_lsns_(const ObTxLogCb *log_cb) int ObPartTransCtx::on_failure(ObTxLogCb *log_cb) { int ret = OB_SUCCESS; - { - CtxLockGuard guard(lock_); - exec_info_.next_log_entry_no_--; - const ObTxLogType log_type = log_cb->get_last_log_type(); - const SCN log_ts = log_cb->get_log_ts(); - // TODO, dingxi - mt_ctx_.sync_log_fail(log_cb->get_callbacks()); - log_cb->get_mds_range().range_sync_failed(mds_cache_); - if (log_ts == ctx_tx_data_.get_start_log_ts()) { - ctx_tx_data_.set_start_log_ts(SCN()); - } - if (ObTxLogTypeChecker::is_state_log(log_type)) { - sub_state_.clear_state_log_submitting(); - } - if (OB_FAIL(fix_redo_lsns_(log_cb))) { - TRANS_LOG(ERROR, "fix redo lsns failed", KR(ret), K(*this)); - } - if (is_contain(log_cb->get_cb_arg_array(), ObTxLogType::TX_BIG_SEGMENT_LOG)) { - remove_unsynced_segment_cb_(log_cb->get_log_ts()); - } - if (ObTxLogType::TX_ROLLBACK_TO_LOG == log_type) { - ObTxData *tx_data = log_cb->get_tx_data(); - if (OB_FAIL(ctx_tx_data_.free_tmp_tx_data(tx_data))) { - TRANS_LOG(WARN, "free tx data failed", KR(ret), K(*this)); - } else { - log_cb->set_tx_data(nullptr); - } - } - if (ObTxLogType::TX_PREPARE_LOG == log_type) { - if (!exec_info_.is_dup_tx_) { - // do nothing - } else if (OB_FAIL(dup_table_before_preapre_(exec_info_.max_applied_log_ts_, true/*before_replay*/))) { - TRANS_LOG(WARN, "set commit_info scn as befre_prepare_version failed", K(ret), KPC(this)); - } else if (OB_FAIL(clear_dup_table_redo_sync_result_())) { - TRANS_LOG(WARN, "clear redo sync result failed", K(ret)); - } - } - if (ObTxLogType::TX_COMMIT_LOG == log_type) { - // if local tx commit log callback on failure, reset trans_version to make standby read skip this - if (is_local_tx_() && !mt_ctx_.get_trans_version().is_max()) { - mt_ctx_.set_trans_version(SCN::max_scn()); - TRANS_LOG(INFO, "clear local trans version when commit log on failure", K(ret), KPC(this)); - } - } - return_log_cb_(log_cb); - log_cb = NULL; - if (ObTxLogType::TX_COMMIT_INFO_LOG == log_type) { - sub_state_.clear_info_log_submitted(); - } - if (busy_cbs_.is_empty() && get_downstream_state() < ObTxState::PREPARE) { - sub_state_.clear_state_log_submitted(); - } - if (busy_cbs_.is_empty() && !has_persisted_log_()) { - // busy callback array is empty and trx has not persisted any log, exit here - TRANS_LOG(WARN, "log sync failed, txn aborted without persisted log", KPC(this)); - if (OB_FAIL(do_local_tx_end_(TxEndAction::ABORT_TX))) { - TRANS_LOG(WARN, "do local tx abort failed", K(ret)); - } - if (need_callback_scheduler_()) { - int tmp_ret = OB_SUCCESS; - if (OB_TMP_FAIL(defer_callback_scheduler_(OB_TRANS_KILLED, SCN::invalid_scn()))) { - TRANS_LOG(WARN, "notify scheduler txn killed fail", K(tmp_ret), K_(trans_id)); - } else { - commit_cb_.disable(); - TRANS_LOG(INFO, "notify scheduler txn killed success", K_(trans_id)); - } - ret = COVER_SUCC(tmp_ret); - } - } - REC_TRANS_TRACE_EXT(tlog_, on_fail_cb, - OB_ID(ret), ret, - OB_ID(log_type), (void*)log_type, - OB_ID(t), log_ts, - OB_ID(ref), get_ref()); - TRANS_LOG(INFO, "ObPartTransCtx::on_failure end", KR(ret), K(*this), KPC(log_cb)); + share::SCN max_committed_scn; + if (OB_FAIL(ls_tx_ctx_mgr_->get_ls_log_adapter()->get_palf_committed_max_scn(max_committed_scn))) { + TRANS_LOG(ERROR, "get palf max committed scn fail, need retry", K(ret)); + ob_abort(); // fast abort for easy debug, TODO(yunxing.cyx) change to return do retry + } else { + TRANS_LOG(INFO, "succ get palf max_commited_scn", K(max_committed_scn), KPC(log_cb)); } - int tmp_ret = OB_SUCCESS; - if (OB_SUCCESS != (tmp_ret = ls_tx_ctx_mgr_->revert_tx_ctx_without_lock(this))) { - TRANS_LOG(ERROR, "release ctx ref failed", KR(tmp_ret)); + if (OB_SUCC(ret)) { + { + const int64_t log_sync_used_time = ObTimeUtility::current_time() - log_cb->get_submit_ts(); + CtxLockGuard guard(lock_); + const int64_t ctx_lock_wait_time = guard.get_lock_acquire_used_time(); + if (log_sync_used_time + ctx_lock_wait_time >= ObServerConfig::get_instance().clog_sync_time_warn_threshold) { + TRANS_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "transaction log sync use too much time", KPC(log_cb), + K(log_sync_used_time), K(ctx_lock_wait_time)); + } + if (log_cb->get_cb_arg_array().count() == 0) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "cb arg array is empty", K(ret), KPC(this)); + print_trace_log_(); + usleep(5000); + ob_abort(); + } + exec_info_.next_log_entry_no_--; + const ObTxLogType log_type = log_cb->get_last_log_type(); + const SCN log_ts = log_cb->get_log_ts(); + mt_ctx_.sync_log_fail(log_cb->get_callbacks(), max_committed_scn); + log_cb->get_mds_range().range_sync_failed(mds_cache_); + if (log_ts == ctx_tx_data_.get_start_log_ts()) { + ctx_tx_data_.set_start_log_ts(SCN()); + } + if (ObTxLogTypeChecker::is_state_log(log_type)) { + sub_state_.clear_state_log_submitting(); + } + if (OB_FAIL(fix_redo_lsns_(log_cb))) { + TRANS_LOG(ERROR, "fix redo lsns failed", KR(ret), K(*this)); + } + if (is_contain(log_cb->get_cb_arg_array(), ObTxLogType::TX_BIG_SEGMENT_LOG)) { + remove_unsynced_segment_cb_(log_cb->get_log_ts()); + } + if (ObTxLogType::TX_ROLLBACK_TO_LOG == log_type) { + ObTxData *tx_data = log_cb->get_tx_data(); + if (OB_FAIL(ctx_tx_data_.free_tmp_tx_data(tx_data))) { + TRANS_LOG(WARN, "free tx data failed", KR(ret), K(*this)); + } else { + log_cb->set_tx_data(nullptr); + } + } + if (ObTxLogType::TX_PREPARE_LOG == log_type) { + if (!exec_info_.is_dup_tx_) { + // do nothing + } else if (OB_FAIL(dup_table_before_preapre_(exec_info_.max_applied_log_ts_, true/*before_replay*/))) { + TRANS_LOG(WARN, "set commit_info scn as befre_prepare_version failed", K(ret), KPC(this)); + } else if (OB_FAIL(clear_dup_table_redo_sync_result_())) { + TRANS_LOG(WARN, "clear redo sync result failed", K(ret)); + } + } + if (ObTxLogType::TX_COMMIT_LOG == log_type) { + // if local tx commit log callback on failure, reset trans_version to make standby read skip this + if (is_local_tx_() && !mt_ctx_.get_trans_version().is_max()) { + mt_ctx_.set_trans_version(SCN::max_scn()); + TRANS_LOG(INFO, "clear local trans version when commit log on failure", K(ret), KPC(this)); + } + } + busy_cbs_.remove(log_cb); + return_log_cb_(log_cb, true); + log_cb = NULL; + if (ObTxLogType::TX_COMMIT_INFO_LOG == log_type) { + sub_state_.clear_info_log_submitted(); + } + if (busy_cbs_.is_empty() && get_downstream_state() < ObTxState::PREPARE) { + sub_state_.clear_state_log_submitted(); + } + if (busy_cbs_.is_empty() && !has_persisted_log_()) { + // busy callback array is empty and trx has not persisted any log, exit here + TRANS_LOG(WARN, "log sync failed, txn aborted without persisted log", KPC(this)); + if (OB_FAIL(do_local_tx_end_(TxEndAction::ABORT_TX))) { + TRANS_LOG(WARN, "do local tx abort failed", K(ret)); + } + if (need_callback_scheduler_()) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(defer_callback_scheduler_(OB_TRANS_KILLED, SCN::invalid_scn()))) { + TRANS_LOG(WARN, "notify scheduler txn killed fail", K(tmp_ret), K_(trans_id)); + } else { + commit_cb_.disable(); + TRANS_LOG(INFO, "notify scheduler txn killed success", K_(trans_id)); + } + ret = COVER_SUCC(tmp_ret); + } + } + REC_TRANS_TRACE_EXT(tlog_, on_fail_cb, + OB_ID(ret), ret, + OB_ID(log_type), (void*)log_type, + OB_ID(t), log_ts, + OB_ID(ref), get_ref()); + TRANS_LOG(INFO, "ObPartTransCtx::on_failure end", KR(ret), K(*this), KPC(log_cb)); + } + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = ls_tx_ctx_mgr_->revert_tx_ctx_without_lock(this))) { + TRANS_LOG(ERROR, "release ctx ref failed", KR(tmp_ret)); + } } return ret; } @@ -2656,284 +2777,88 @@ int ObPartTransCtx::generate_commit_version_() return ret; } -int ObPartTransCtx::fill_redo_log_(char *buf, - const int64_t buf_len, - int64_t &pos, - ObRedoLogSubmitHelper &helper) -{ - int ret = OB_SUCCESS; - const bool log_for_lock_node = true; - // const bool log_for_lock_node = !(is_local_tx_() && (part_trans_action_ == ObPartTransAction::COMMIT)); - // // // // - // Possible problems caused: - // 1. block frozen : We wiil not add data_node_count if skip lock node in fill_redo. Then the next memtable - // will return -4112 and the prev memtable will wait for lock node clearing unsubmitted_cnt. - // 2. replay checksum error: - // | - // | part_trans_action != COMMIT - // v - // +-------------------------------------+ - // | start submit_commit_log | - // +-------------------------------------+ - // | - // | - // v - // +-------------------------------------+ - // | log_for_lock_node == false | - // | cal checksum with lock node | - // +-------------------------------------+ - // | - // | - // v - // +-------------------------------------+ - // | submit commit log failed with -4038 | - // | rewrite as OB_SUCCESS | - // +-------------------------------------+ - // | - // | part_trans_action = COMMIT - // v - // +-------------------------------------+ - // | retry submit commit log | - // +-------------------------------------+ - // | - // | - // v - // +-------------------------------------+ - // | replay checksum ERROR | - // +-------------------------------------+ - - if (OB_UNLIKELY(NULL == buf || buf_len < 0 || pos < 0 || buf_len < pos)) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid argument", KR(ret), KP(buf), K(buf_len), K(pos), K(*this)); - } else if (OB_SUCCESS - != (ret = mt_ctx_.fill_redo_log(buf, buf_len, pos, helper, log_for_lock_node))) { - if (OB_EAGAIN != ret && OB_ENTRY_NOT_EXIST != ret) { - TRANS_LOG(WARN, "fill redo log failed", KR(ret), K(*this)); - } - } - - return ret; -} - // starting from 0 int64_t ObPartTransCtx::get_redo_log_no_() const { return exec_info_.redo_lsns_.count(); } -int ObPartTransCtx::submit_redo_log_(ObTxLogBlock &log_block, - bool &has_redo, - ObRedoLogSubmitHelper &helper) + +inline +int ObPartTransCtx::submit_redo_if_serial_logging_(ObTxLogBlock &log_block, + bool &has_redo, + ObRedoLogSubmitHelper &helper) { int ret = OB_SUCCESS; - int64_t mutator_size = 0; - bool need_continue = true; - bool need_submit_log = false; - bool need_undo_log = false; - ObTxLogCb *log_cb = NULL; - bool need_stop_submit_redo = false; - - while (OB_SUCC(ret) && need_continue) { - ObTxRedoLog redo_log(get_redo_log_no_(), cluster_version_); - mutator_size = 0; - need_undo_log = false; - need_submit_log = false; - log_cb = NULL; - helper.reset(); - - if (OB_FAIL(exec_info_.redo_lsns_.reserve(exec_info_.redo_lsns_.count() + 1))) { - TRANS_LOG(WARN, "reserve memory for redo lsn failed", K(ret)); - } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { - if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { - TRANS_LOG(WARN, "get log cb failed", KR(ret), K(*this)); - } - } else if (OB_FAIL(log_block.prepare_mutator_buf(redo_log))) { - TRANS_LOG(WARN, "prepare mutator buf failed", KR(ret)); - return_log_cb_(log_cb); - log_cb = NULL; - } else { - ret = fill_redo_log_(redo_log.get_mutator_buf(), redo_log.get_mutator_size(), mutator_size, helper); - if (OB_SUCCESS == ret) { - has_redo = true; - if (OB_FAIL(log_block.finish_mutator_buf(redo_log, mutator_size))) { - TRANS_LOG(WARN, "finish mutator buf failed", KR(ret), K(*this)); - } else if (OB_FAIL(log_block.add_new_log(redo_log))) { - TRANS_LOG(WARN, "add redo log failed", KR(ret), K(*this)); - } else { - need_continue = false; - } - if (OB_FAIL(ret)) { - return_log_cb_(log_cb); - log_cb = NULL; - need_undo_log = true; - } - } else if (OB_EAGAIN == ret) { - has_redo = true; - /* - * stop submitting redo log when someone is waitting for tx lock - * e.g. large tranction submit too many redo log for a long time - * avoid others wait for ctx lock too long - * and the submitted redo log callback will drive remain log submit - */ - need_stop_submit_redo = lock_.get_waiting_lock_cnt() > 0; - if (OB_FAIL(log_block.finish_mutator_buf(redo_log, mutator_size))) { - TRANS_LOG(WARN, "finish mutator buf failed", KR(ret), K(*this)); - } else if (OB_FAIL(log_block.add_new_log(redo_log))) { - TRANS_LOG(WARN, "add redo log failed", KR(ret), K(*this)); - } else { - need_submit_log = true; - } - if (OB_FAIL(ret)) { - return_log_cb_(log_cb); - log_cb = NULL; - need_undo_log = true; - } - } else if (OB_ENTRY_NOT_EXIST == ret || OB_BLOCK_FROZEN == ret) { - TRANS_LOG(TRACE, "no redo log to be flushed", KR(ret), K(*this)); - // rewrite ret - ret = (OB_ENTRY_NOT_EXIST == ret) ? OB_SUCCESS : ret; - has_redo = false; - if (OB_SUCC(ret) && OB_FAIL(log_block.finish_mutator_buf(redo_log, 0))) { - TRANS_LOG(WARN, "finish mutator buf failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = NULL; - } - need_continue = false; - } else if (OB_ERR_TOO_BIG_ROWSIZE == ret) { - TRANS_LOG(WARN, "encounter too big row size error, need extend log buffer", K(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = NULL; - const int save_ret = ret; - // rewrite ret - ret = OB_SUCCESS; - if (OB_FAIL(log_block.finish_mutator_buf(redo_log, 0))) { - TRANS_LOG(WARN, "finish mutator buf failed", KR(ret), K(*this)); - } else if (OB_FAIL(log_block.extend_log_buf())) { - TRANS_LOG(WARN, "extend log buffer failed", K(ret), K(*this)); - if (OB_ALLOCATE_MEMORY_FAILED != ret) { - ret = save_ret; - } - } else { - TRANS_LOG(INFO, "extend log buffer success", K(*this), K(log_block)); - } - } else { - TRANS_LOG(WARN, "fill redo log failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = NULL; - } - } - if (need_submit_log) { - if (OB_FAIL(acquire_ctx_ref_())) { - TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { - TRANS_LOG(WARN, "submit log failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = NULL; - need_undo_log = true; - release_ctx_ref_(); - } else if (OB_FAIL(after_submit_log_(log_block, log_cb, &helper))) { - } else { - // TRANS_LOG(INFO, "submit redo log in clog adapter success", K(*log_cb)); - has_redo = false; - } - } else { - if (NULL != log_cb) { - return_log_cb_(log_cb); - log_cb = NULL; - } - } - - if (need_undo_log) { - has_redo = false; - } - - if (OB_SUCC(ret) && need_continue && need_stop_submit_redo) { - ret = OB_EAGAIN; + if (OB_LIKELY(!is_parallel_logging())) { + ObTxRedoSubmitter submitter(*this, mt_ctx_); + ret = submitter.fill(log_block, helper, true /*display blocked info*/); + has_redo = submitter.get_submitted_cnt() > 0 || helper.callbacks_.count() > 0; + } else { + // sanity check, all redo must have been flushed #ifndef NDEBUG - TRANS_LOG(INFO, "stop submitting redo log", KR(ret), K(*this)); + mt_ctx_.check_all_redo_flushed(); #endif - } } - return ret; } -int ObPartTransCtx::submit_redo_log_() +// when parallel logging, redo need submitted seperate with other txn's log +inline +int ObPartTransCtx::submit_redo_if_parallel_logging_() { int ret = OB_SUCCESS; - ObTxLogBlock log_block; - ObTxLogCb *log_cb = NULL; - bool has_redo = false; - const int64_t replay_hint = static_cast(trans_id_.get_id()); - ObRedoLogSubmitHelper helper; - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { - TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_redo_log_(log_block, has_redo, helper))) { - // this function is called by freezing and instant logging, - // don't need to handle OB_BLOCK_FROZEN ret - if (OB_BLOCK_FROZEN == ret) { - TRANS_LOG(INFO, "submit log meets frozen memtable", KR(ret), K(*this)); - } else if (OB_EAGAIN == ret) { -#ifndef NDEBUG - TRANS_LOG(INFO, "stop submitting redo log", KR(ret), K(*this)); -#endif - } else if (REACH_TIME_INTERVAL(100 * 1000)) { - TRANS_LOG(WARN, "submit redo log failed", KR(ret), K(*this)); + if (OB_UNLIKELY(is_parallel_logging())) { + ObTxRedoSubmitter submitter(*this, mt_ctx_); + if (OB_FAIL(submitter.submit(true/*flush all*/, false/*final serial*/, true /*display blocked info*/))) { + TRANS_LOG(WARN, "submit redo log fail", K(ret)); } - } else if (!has_redo) { - // do nothing - } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { - if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { - TRANS_LOG(WARN, "get log cb failed", KR(ret), K(*this)); - } - } else if (log_block.get_cb_arg_array().count() == 0) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); - return_log_cb_(log_cb); - log_cb = NULL; - } else if (OB_FAIL(acquire_ctx_ref_())) { - TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { - TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = NULL; - release_ctx_ref_(); - } else if (OB_FAIL(after_submit_log_(log_block, log_cb, &helper))) { - } else { - // TRANS_LOG(INFO, "submit redolog in clog adapter success", K(*log_cb)); - log_cb = NULL; } - return ret; } +// this function is thread safe, not need other lock's protection +inline +int ObPartTransCtx::init_log_block_(ObTxLogBlock &log_block, + const int64_t suggested_buf_size, + const bool serial_final) +{ + ObTxLogBlockHeader &header = log_block.get_header(); + // the log_entry_no will be backfill before log-block to be submitted + header.init(cluster_id_, cluster_version_, -1 /*log_entry_no*/, trans_id_, exec_info_.scheduler_); + if (OB_UNLIKELY(serial_final)) { header.set_serial_final(); } + return log_block.init_for_fill(suggested_buf_size); +} + +inline int ObPartTransCtx::reuse_log_block_(ObTxLogBlock &log_block) +{ + ObTxLogBlockHeader &header = log_block.get_header(); + header.init(cluster_id_, cluster_version_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); + return log_block.reuse_for_fill(); +} + int ObPartTransCtx::submit_redo_commit_info_log_() { int ret = OB_SUCCESS; ObTxLogBlock log_block; bool has_redo = false; ObTxLogCb *log_cb = NULL; - const int64_t replay_hint = static_cast(trans_id_.get_id()); ObRedoLogSubmitHelper helper; - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - + const int64_t replay_hint = trans_id_.get_id(); + using LogBarrierType = logservice::ObReplayBarrierType; + LogBarrierType barrier = LogBarrierType::NO_NEED_BARRIER; if (need_force_abort_() || is_force_abort_logging_() || get_downstream_state() == ObTxState::ABORT) { ret = OB_TRANS_KILLED; TRANS_LOG(WARN, "tx has been aborting, can not submit prepare log", K(ret)); } else if (sub_state_.is_info_log_submitted()) { // state log already submitted, do nothing - } else if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + } else if (OB_FAIL(submit_redo_if_parallel_logging_())) { + } else if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); } else if (OB_FAIL(submit_multi_data_source_(log_block))) { TRANS_LOG(WARN, "submit multi source data failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_redo_commit_info_log_(log_block, has_redo, helper))) { + } else if (OB_FAIL(submit_redo_commit_info_log_(log_block, has_redo, helper, barrier))) { TRANS_LOG(WARN, "submit redo commit state log failed", KR(ret), K(*this)); } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { @@ -2944,10 +2869,13 @@ int ObPartTransCtx::submit_redo_commit_info_log_() TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb, replay_hint, barrier))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -2977,24 +2905,22 @@ int ObPartTransCtx::validate_commit_info_log_(const ObTxCommitInfoLog &commit_in int ObPartTransCtx::submit_redo_commit_info_log_(ObTxLogBlock &log_block, bool &has_redo, - ObRedoLogSubmitHelper &helper) + ObRedoLogSubmitHelper &helper, + logservice::ObReplayBarrierType &barrier) { int ret = OB_SUCCESS; ObTxLogCb *log_cb = NULL; - - logservice::ObReplayBarrierType commit_info_log_barrier_type = - logservice::ObReplayBarrierType::NO_NEED_BARRIER; + const int64_t replay_hint = trans_id_.get_id(); + barrier = logservice::ObReplayBarrierType::NO_NEED_BARRIER; if (sub_state_.is_info_log_submitted()) { // state log already submitted, do nothing - } else if (OB_FAIL(submit_redo_log_(log_block, has_redo, helper))) { + } else if (OB_FAIL(submit_redo_if_serial_logging_(log_block, has_redo, helper))) { TRANS_LOG(WARN, "submit redo log failed", KR(ret), K(*this)); } else if (OB_FAIL(check_dup_trx_with_submitting_all_redo(log_block, helper))) { TRANS_LOG(WARN, "check dup trx with submitting all redo failed", K(ret)); - } else if (OB_FAIL(decide_state_log_barrier_type_(ObTxLogType::TX_COMMIT_INFO_LOG, - commit_info_log_barrier_type))) { - TRANS_LOG(WARN, "decide commit info log barrier failed", K(ret), - K(commit_info_log_barrier_type), KPC(this)); + } else if (OB_FAIL(decide_state_log_barrier_type_(ObTxLogType::TX_COMMIT_INFO_LOG, barrier))) { + TRANS_LOG(WARN, "decide commit info log barrier failed", K(ret), K(barrier), KPC(this)); } else { ObTxCommitInfoLog commit_info_log( exec_info_.scheduler_, exec_info_.participants_, exec_info_.upstream_, @@ -3027,11 +2953,13 @@ int ObPartTransCtx::submit_redo_commit_info_log_(ObTxLogBlock &log_block, TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; - // acquire ctx ref before submit log + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb))) { TRANS_LOG(WARN, "submit log failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3044,23 +2972,12 @@ int ObPartTransCtx::submit_redo_commit_info_log_(ObTxLogBlock &log_block, K(ls_id_)); } else if (OB_FAIL(log_block.add_new_log(commit_info_log))) { TRANS_LOG(WARN, "add new log failed", KR(ret), K(*this)); - } else if (commit_info_log_barrier_type - != logservice::ObReplayBarrierType::NO_NEED_BARRIER - && OB_FAIL(log_block.rewrite_barrier_log_block( - trans_id_.get_id(), commit_info_log_barrier_type))) { - TRANS_LOG(WARN, "rewrite commit info log barrier type failed", K(ret), - K(commit_info_log_barrier_type), KPC(this)); } has_redo = false; } } else { TRANS_LOG(WARN, "add new log failed", KR(ret), K(this)); } - } else if (commit_info_log_barrier_type != logservice::ObReplayBarrierType::NO_NEED_BARRIER - && OB_FAIL(log_block.rewrite_barrier_log_block(trans_id_.get_id(), - commit_info_log_barrier_type))) { - TRANS_LOG(WARN, "rewrite commit info log barrier type failed", K(ret), - K(commit_info_log_barrier_type), KPC(this)); } } @@ -3072,19 +2989,21 @@ int ObPartTransCtx::submit_redo_active_info_log_() int ret = OB_SUCCESS; ObTxLogBlock log_block; bool has_redo = false; - const int64_t replay_hint = static_cast(trans_id_.get_id()); ObRedoLogSubmitHelper helper; - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + if (OB_FAIL(submit_redo_if_parallel_logging_())) { + } else if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); } else if (OB_FAIL(submit_multi_data_source_(log_block))) { TRANS_LOG(WARN, "submit multi source data failed", K(ret), K(*this)); - } else if (OB_FAIL(submit_redo_log_(log_block, has_redo, helper))) { + } else if (OB_FAIL(submit_redo_if_serial_logging_(log_block, has_redo, helper))) { TRANS_LOG(WARN, "submit redo log failed", KR(ret), K(*this)); + } else if (OB_FAIL(submit_pending_log_block_(log_block, helper, logservice::ObReplayBarrierType::NO_NEED_BARRIER))) { + TRANS_LOG(WARN, "submit pending log failed", KR(ret), K(*this)); + } else if (exec_info_.redo_lsns_.count() > 0 && OB_FAIL(submit_record_log_())) { + TRANS_LOG(WARN, "submit record log failed", KR(ret), K(*this)); + } else if (OB_FAIL(reuse_log_block_(log_block))) { + TRANS_LOG(WARN, "reuse log block failed", KR(ret), K(*this)); } else { - ObTxSEQ cur_submitted_seq_no = MAX(exec_info_.max_submitted_seq_no_, helper.max_seq_no_); ObTxActiveInfoLog active_info_log(exec_info_.scheduler_, exec_info_.trans_type_, session_id_, trace_info_.get_app_trace_id(), mt_ctx_.get_min_table_version(), can_elr_, @@ -3093,64 +3012,17 @@ int ObPartTransCtx::submit_redo_active_info_log_() false, // sub2pc exec_info_.is_dup_tx_, trans_expired_time_, epoch_, last_op_sn_, first_scn_, - last_scn_, cur_submitted_seq_no, + last_scn_, exec_info_.max_submitted_seq_no_, cluster_version_, - exec_info_.xid_); - bool redo_log_submitted = false; + exec_info_.xid_, + exec_info_.serial_final_seq_no_); ObTxLogCb *log_cb = nullptr; if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { TRANS_LOG(WARN, "get log cb failed", KR(ret), KP(log_cb), K(*this)); } else if (OB_FAIL(log_block.add_new_log(active_info_log))) { - if (OB_BUF_NOT_ENOUGH == ret) { - TRANS_LOG(WARN, "buf not enough", K(ret), K(active_info_log)); - if (log_block.get_cb_arg_array().count() == 0) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); - return_log_cb_(log_cb); - log_cb = NULL; - } else if (OB_FAIL(acquire_ctx_ref_())) { - TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { - TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = nullptr; - release_ctx_ref_(); - } else if (OB_FAIL(after_submit_log_(log_block, log_cb, &helper))) { - } else { - redo_log_submitted = true; - // TRANS_LOG(INFO, "submit redo log success", K(*log_cb)); - log_cb = nullptr; - if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { - TRANS_LOG(WARN, "get log cb failed", KR(ret), KP(log_cb), K(*this)); - } else if (OB_FAIL(log_block.add_new_log(active_info_log))) { - TRANS_LOG(WARN, "add new log failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = nullptr; - } else if (log_block.get_cb_arg_array().count() == 0) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); - return_log_cb_(log_cb); - log_cb = NULL; - } else if (OB_FAIL(acquire_ctx_ref_())) { - TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { - TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = nullptr; - release_ctx_ref_(); - } else if (OB_FAIL(after_submit_log_(log_block, log_cb, NULL))) { - } else { - TRANS_LOG(INFO, "submit active info success", K(*log_cb)); - log_cb = nullptr; - } - } - } else { - TRANS_LOG(WARN, "add new log failed", KR(ret), K(*this)); - return_log_cb_(log_cb); - log_cb = nullptr; - } + TRANS_LOG(WARN, "add new log failed", KR(ret), K(*this)); + return_log_cb_(log_cb); + log_cb = nullptr; } else if (log_block.get_cb_arg_array().count() == 0) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); @@ -3158,8 +3030,7 @@ int ObPartTransCtx::submit_redo_active_info_log_() log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = nullptr; @@ -3181,36 +3052,43 @@ int ObPartTransCtx::submit_prepare_log_() palf::LSN prev_lsn; bool has_redo = false; bool redo_log_submitted = false; - const int64_t replay_hint = static_cast(trans_id_.get_id()); ObRedoLogSubmitHelper helper; - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - + const int64_t replay_hint = trans_id_.get_id(); + using ReplayBarrier = logservice::ObReplayBarrierType; + ReplayBarrier commit_info_log_barrier = ReplayBarrier::NO_NEED_BARRIER; if (need_force_abort_() || is_force_abort_logging_() || get_downstream_state() == ObTxState::ABORT) { ret = OB_TRANS_KILLED; TRANS_LOG(WARN, "tx has been aborting, can not submit prepare log", K(ret)); } - + bool contain_commit_info = false; if (OB_SUCC(ret)) { - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { - TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); - } else if (!sub_state_.is_info_log_submitted()) { + if (!sub_state_.is_info_log_submitted()) { prev_lsn.reset(); - if (OB_FAIL(submit_multi_data_source_(log_block))) { + if (OB_FAIL(submit_redo_if_parallel_logging_())) { + } else if (OB_FAIL(init_log_block_(log_block))) { + TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); + } else if (OB_FAIL(submit_multi_data_source_(log_block))) { TRANS_LOG(WARN, "submit multi source data failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_redo_commit_info_log_(log_block, has_redo, helper))) { + } else if (OB_FAIL(submit_redo_commit_info_log_(log_block, has_redo, helper, + commit_info_log_barrier))) { TRANS_LOG(WARN, "submit redo commit state log failed", KR(ret), K(*this)); } else { // do nothing + contain_commit_info = true; } + // init log_block for prepare log + } else if (OB_FAIL(init_log_block_(log_block))) { + TRANS_LOG(WARN, "init log block failed", K(ret)); } } + const ReplayBarrier compound_log_barrier = + contain_commit_info ? commit_info_log_barrier : ReplayBarrier::NO_NEED_BARRIER; if (OB_SUCC(ret)) { if (exec_info_.is_dup_tx_ && !is_dup_table_redo_sync_completed_()) { - if (OB_FAIL(submit_pending_log_block_(log_block, helper))) { + if (OB_FAIL(submit_pending_log_block_(log_block, helper, compound_log_barrier))) { TRANS_LOG(WARN, "submit pending log block failed", K(ret)); } else { ret = OB_EAGAIN; @@ -3220,7 +3098,6 @@ int ObPartTransCtx::submit_prepare_log_() } } - if (OB_SUCC(ret)) { if (OB_FAIL(errism_submit_prepare_log_())) { TRANS_LOG(WARN, "errsim for submit prepare log", K(ret), KPC(this)); @@ -3250,10 +3127,14 @@ int ObPartTransCtx::submit_prepare_log_() TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb, replay_hint, + compound_log_barrier))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3279,9 +3160,7 @@ int ObPartTransCtx::submit_prepare_log_() log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), exec_info_.prepare_version_, - log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, exec_info_.prepare_version_, log_cb))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3302,11 +3181,13 @@ int ObPartTransCtx::submit_prepare_log_() TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), exec_info_.prepare_version_, log_cb, - false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, exec_info_.prepare_version_, log_cb, replay_hint, compound_log_barrier))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3329,13 +3210,12 @@ int ObPartTransCtx::submit_commit_log_() ObTxLogBlock log_block; palf::LSN prev_lsn; bool has_redo = false; - const int64_t replay_hint = static_cast(trans_id_.get_id()); ObRedoLogSubmitHelper helper; ObTxBufferNodeArray multi_source_data; - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); + const int64_t replay_hint = trans_id_.get_id(); const bool local_tx = is_local_tx_(); - + using LogBarrierType = logservice::ObReplayBarrierType; + LogBarrierType commit_info_log_barrier = LogBarrierType::NO_NEED_BARRIER; if (need_force_abort_() || is_force_abort_logging_() || get_downstream_state() == ObTxState::ABORT) { ret = OB_TRANS_KILLED; @@ -3343,6 +3223,7 @@ int ObPartTransCtx::submit_commit_log_() } else if (OB_FAIL(gen_final_mds_array_(multi_source_data))) { TRANS_LOG(WARN, "gen total multi source data failed", KR(ret), K(*this)); } else { + bool log_block_inited = false; int64_t suggested_buf_size = ObTxAdaptiveLogBuf::NORMAL_LOG_BUF_SIZE; if (local_tx && multi_source_data.count() == 0 && @@ -3352,25 +3233,32 @@ int ObPartTransCtx::submit_commit_log_() IS_CORNER(10000))) { suggested_buf_size = ObTxAdaptiveLogBuf::MIN_LOG_BUF_SIZE; } - if (OB_FAIL(log_block.init(replay_hint, log_block_header, suggested_buf_size))) { - TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); - } else if (local_tx) { + if (local_tx) { if (!sub_state_.is_info_log_submitted()) { prev_lsn.reset(); - if (OB_FAIL(submit_multi_data_source_(log_block))) { + if (OB_FAIL(submit_redo_if_parallel_logging_())) { + } else if (OB_FAIL(init_log_block_(log_block, suggested_buf_size))) { + TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); + } else if (FALSE_IT(log_block_inited = true)) { + } else if (OB_FAIL(submit_multi_data_source_(log_block))) { TRANS_LOG(WARN, "submit multi source data failed", KR(ret), K(*this)); - } else if (OB_SUCC(submit_redo_commit_info_log_(log_block, has_redo, helper))) { + } else if (OB_SUCC(submit_redo_commit_info_log_(log_block, has_redo, helper, + commit_info_log_barrier))) { // do nothing } else { TRANS_LOG(WARN, "submit redo commit state log failed", KR(ret), K(*this)); } } } + // init log_block for commit log + if (OB_SUCC(ret) && !log_block_inited && OB_FAIL(init_log_block_(log_block, suggested_buf_size))) { + TRANS_LOG(WARN, "init log block failed", K(ret)); + } } if (OB_SUCC(ret)) { if (exec_info_.is_dup_tx_ && !is_dup_table_redo_sync_completed_()) { - if (OB_FAIL(submit_pending_log_block_(log_block, helper))) { + if (OB_FAIL(submit_pending_log_block_(log_block, helper, commit_info_log_barrier))) { TRANS_LOG(WARN, "submit pending log block failed", K(ret)); } else { ret = OB_EAGAIN; @@ -3381,10 +3269,13 @@ int ObPartTransCtx::submit_commit_log_() } if (OB_SUCC(ret)) { - const uint64_t checksum = - (exec_info_.need_checksum_ && !is_incomplete_replay_ctx_ ? mt_ctx_.calc_checksum_all() : 0); SCN log_commit_version; - if (!local_tx) { + ObSEArray checksum_arr; + if (exec_info_.need_checksum_ + && replay_completeness_.is_complete() + && OB_FAIL(mt_ctx_.calc_checksum_all(checksum_arr))) { + TRANS_LOG(WARN, "calc checksum failed", K(ret)); + } else if (!local_tx) { log_commit_version = ctx_tx_data_.get_commit_version(); if (OB_FAIL(get_prev_log_lsn_(log_block, ObTxLogType::TX_PREPARE_LOG, prev_lsn))) { TRANS_LOG(WARN, "get prev log lsn failed", K(ret), K(*this)); @@ -3397,24 +3288,31 @@ int ObPartTransCtx::submit_commit_log_() TRANS_LOG(WARN, "get prev log lsn failed", K(ret), K(*this)); } } - - ObTxCommitLog commit_log(log_commit_version, checksum, exec_info_.incremental_participants_, + uint64_t collapsed_checksum = 0; + uint8_t _checksum_sig[checksum_arr.count()]; + ObArrayHelper checksum_sig(checksum_arr.count(), _checksum_sig); + mt_ctx_.convert_checksum_for_commit_log(checksum_arr, collapsed_checksum, checksum_sig); + ObTxCommitLog commit_log(log_commit_version, + collapsed_checksum, + checksum_sig, + exec_info_.incremental_participants_, multi_source_data, exec_info_.trans_type_, prev_lsn, coord_prepare_info_arr_); ObTxLogCb *log_cb = NULL; bool redo_log_submitted = false; - logservice::ObReplayBarrierType commit_log_barrier_type = logservice::ObReplayBarrierType::NO_NEED_BARRIER; - + LogBarrierType commit_log_barrier_type = LogBarrierType::NO_NEED_BARRIER; + LogBarrierType compound_log_barrier_type = commit_info_log_barrier; if (OB_SUCC(ret)) { if (OB_FAIL(set_start_scn_in_commit_log_(commit_log))) { TRANS_LOG(WARN, "set start scn in commit log failed", K(ret), K(commit_log)); } else if ((exec_info_.multi_data_source_.count() > 0 || mds_cache_.count() > 0) && OB_FAIL(try_alloc_retain_ctx_func_())) { TRANS_LOG(WARN, "alloc retain ctx func for mds trans failed", K(ret), K(mds_cache_), KPC(this)); - } else if (OB_FAIL(decide_state_log_barrier_type_(ObTxLogType::TX_COMMIT_LOG, - commit_log_barrier_type))) { + } else if (OB_FAIL(decide_state_log_barrier_type_(ObTxLogType::TX_COMMIT_LOG, commit_log_barrier_type))) { TRANS_LOG(WARN, "decide commit log barrier type failed", K(ret), K(commit_log_barrier_type), KPC(this)); + } else if (OB_FAIL(ObTxLogTypeChecker::decide_final_barrier_type(commit_log_barrier_type, compound_log_barrier_type))) { + TRANS_LOG(ERROR, "decide compound log barrier type failed", K(ret), K(commit_log_barrier_type), K(compound_log_barrier_type), KPC(this)); } } @@ -3435,11 +3333,14 @@ int ObPartTransCtx::submit_commit_log_() TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; - // acquire ctx ref before submit log + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb, replay_hint, + commit_info_log_barrier))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3467,11 +3368,6 @@ int ObPartTransCtx::submit_commit_log_() TRANS_LOG(WARN, "add new log failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; - } else if (commit_log_barrier_type != logservice::ObReplayBarrierType::NO_NEED_BARRIER - && OB_FAIL(log_block.rewrite_barrier_log_block(trans_id_.get_id(), commit_log_barrier_type))) { - TRANS_LOG(WARN, "rewrite commit log barrier failed", K(ret)); - return_log_cb_(log_cb); - log_cb = NULL; } else if (log_block.get_cb_arg_array().count() == 0) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); @@ -3479,9 +3375,11 @@ int ObPartTransCtx::submit_commit_log_() log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), - ctx_tx_data_.get_commit_version(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, + ctx_tx_data_.get_commit_version(), + log_cb, + replay_hint, + commit_log_barrier_type))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3505,11 +3403,6 @@ int ObPartTransCtx::submit_commit_log_() return_log_cb_(log_cb); log_cb = NULL; } - } else if (commit_log_barrier_type != logservice::ObReplayBarrierType::NO_NEED_BARRIER - && OB_FAIL(log_block.rewrite_barrier_log_block(trans_id_.get_id(), commit_log_barrier_type))) { - TRANS_LOG(WARN, "rewrite commit log barrier failed", K(ret)); - return_log_cb_(log_cb); - log_cb = NULL; } else if (log_block.get_cb_arg_array().count() == 0) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); @@ -3519,11 +3412,17 @@ int ObPartTransCtx::submit_commit_log_() if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { TRANS_LOG(WARN, "get log cb failed", KR(ret), K(*this)); } + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), ctx_tx_data_.get_commit_version(), - log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, + ctx_tx_data_.get_commit_version(), + log_cb, + replay_hint, + compound_log_barrier_type))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); release_ctx_ref_(); return_log_cb_(log_cb); @@ -3553,12 +3452,9 @@ int ObPartTransCtx::submit_abort_log_() ObTxLogCb *log_cb = NULL; ObTxLogBlock log_block; ObTxBufferNodeArray tmp_array; - - logservice::ObReplayBarrierType abort_log_barrier_type = - logservice::ObReplayBarrierType::NO_NEED_BARRIER; - const int64_t replay_hint = static_cast(trans_id_.get_id()); - ObTxLogBlockHeader log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, - exec_info_.scheduler_); + const int64_t replay_hint = trans_id_.get_id(); + using LogBarrierType = logservice::ObReplayBarrierType; + logservice::ObReplayBarrierType barrier = LogBarrierType::NO_NEED_BARRIER; if (OB_FAIL(gen_final_mds_array_(tmp_array, false))) { TRANS_LOG(WARN, "gen abort mds array failed", K(ret)); @@ -3569,12 +3465,13 @@ int ObPartTransCtx::submit_abort_log_() if (OB_SUCC(ret)) { if ((exec_info_.multi_data_source_.count() > 0 || mds_cache_.count() > 0)) { if (OB_FAIL(try_alloc_retain_ctx_func_())) { - TRANS_LOG(WARN, "alloc retain ctx func for mds trans failed", K(ret), K(mds_cache_), - KPC(this)); - } else if (OB_FAIL(decide_state_log_barrier_type_(ObTxLogType::TX_ABORT_LOG, - abort_log_barrier_type))) { - TRANS_LOG(WARN, "decide abort log barrier type failed", K(ret), K(abort_log_barrier_type), - KPC(this)); + TRANS_LOG(WARN, "alloc retain ctx func for mds trans failed", K(ret), K(mds_cache_), KPC(this)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(decide_state_log_barrier_type_(ObTxLogType::TX_ABORT_LOG, barrier))) { + TRANS_LOG(WARN, "decide abort log barrier type failed", K(ret), K(barrier), KPC(this)); } } @@ -3591,15 +3488,10 @@ int ObPartTransCtx::submit_abort_log_() } if (OB_FAIL(ret)) { - } else if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + } else if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); } else if (OB_FAIL(log_block.add_new_log(abort_log))) { TRANS_LOG(WARN, "add new log failed", KR(ret), K(*this)); - } else if (abort_log_barrier_type != logservice::ObReplayBarrierType::NO_NEED_BARRIER - && OB_FAIL( - log_block.rewrite_barrier_log_block(trans_id_.get_id(), abort_log_barrier_type))) { - TRANS_LOG(WARN, "rewrite barrier log block failed", K(ret), K(abort_log_barrier_type), - KPC(this)); } else if (log_block.get_cb_arg_array().count() == 0) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); @@ -3611,8 +3503,7 @@ int ObPartTransCtx::submit_abort_log_() } } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb, replay_hint, barrier))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3631,12 +3522,9 @@ int ObPartTransCtx::submit_clear_log_() int ret = OB_SUCCESS; ObTxLogBlock log_block; ObTxClearLog clear_log(exec_info_.incremental_participants_); + const int64_t replay_hint = trans_id_.get_id(); ObTxLogCb *log_cb = NULL; - const int64_t replay_hint = static_cast(trans_id_.get_id()); - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { @@ -3653,10 +3541,9 @@ int ObPartTransCtx::submit_clear_log_() log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), - share::SCN::max(ctx_tx_data_.get_end_log_ts(), max_2pc_commit_scn_), log_cb, - false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, + share::SCN::max(ctx_tx_data_.get_end_log_ts(), max_2pc_commit_scn_), + log_cb))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3675,12 +3562,9 @@ int ObPartTransCtx::submit_record_log_() int ret = OB_SUCCESS; ObTxLogBlock log_block; ObTxRecordLog record_log(exec_info_.prev_record_lsn_, exec_info_.redo_lsns_); + const int64_t replay_hint = trans_id_.get_id(); ObTxLogCb *log_cb = NULL; - const int64_t replay_hint = static_cast(trans_id_.get_id()); - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { @@ -3697,8 +3581,7 @@ int ObPartTransCtx::submit_record_log_() log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3729,21 +3612,19 @@ int ObPartTransCtx::submit_big_segment_log_() // if one part of big segment log submit into palf failed , the transaction must drive into abort // phase. + if (OB_FAIL(init_log_block_(log_block))) { + TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); + } while (OB_SUCC(ret) && big_segment_info_.segment_buf_.is_active()) { const char *submit_buf = nullptr; int64_t submit_buf_len = 0; - ObTxLogBlockHeader log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { - TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); - } else if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { + if (OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { TRANS_LOG(WARN, "get log cb failed", KR(ret), K(*this)); } - } else if (OB_FALSE_IT(log_cb->copy(*big_segment_info_.submit_log_cb_template_))) { - - } else if (OB_FALSE_IT(ret = (log_block.acquire_segment_log_buf( - submit_buf, submit_buf_len, log_block_header, source_log_type, - &big_segment_info_.segment_buf_)))) { + } else if (OB_FAIL(log_cb->copy(*big_segment_info_.submit_log_cb_template_))) { + TRANS_LOG(WARN, "log cb copy failed", KR(ret), K(*this)); + } else if (OB_FALSE_IT(ret = (log_block.acquire_segment_log_buf(source_log_type, &big_segment_info_.segment_buf_)))) { } else if (OB_EAGAIN != ret && OB_ITER_END != ret) { TRANS_LOG(WARN, "acquire one part of big segment log failed", KR(ret), K(*this)); return_log_cb_(log_cb); @@ -3757,9 +3638,7 @@ int ObPartTransCtx::submit_big_segment_log_() log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - submit_buf, submit_buf_len, big_segment_info_.submit_base_scn_, log_cb, - false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, big_segment_info_.submit_base_scn_, log_cb))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -3874,6 +3753,41 @@ share::SCN ObPartTransCtx::get_min_unsyncd_segment_scn_() return min_scn; } +inline +int ObPartTransCtx::submit_log_block_out_(ObTxLogBlock &log_block, + const share::SCN &base_scn, + ObTxLogCb *&log_cb, + const int64_t replay_hint, + const logservice::ObReplayBarrierType barrier) +{ + int ret = OB_SUCCESS; + bool is_2pc_state_log = false; + if (OB_UNLIKELY(is_2pc_blocking())) { + ret = OB_EAGAIN; + TRANS_LOG(WARN, "tx submit log failed because of 2pc blocking", K(ret), KPC(this)); + // It is safe to merge the intermediate_participants because we will block + // the persistent state machine with is_2pc_blocking. The detailed design + // can be found in the implementation of the merge_intermediate_participants. + } else if (is_contain_stat_log(log_block.get_cb_arg_array()) && FALSE_IT(is_2pc_state_log = true)) { + } else if (is_2pc_state_log && OB_FAIL(merge_intermediate_participants())) { + TRANS_LOG(WARN, "fail to merge intermediate participants", K(ret), KPC(this)); + } else { + const int64_t replay_hint_v = replay_hint ?: trans_id_.get_id(); + log_block.get_header().set_log_entry_no(exec_info_.next_log_entry_no_); + if (OB_FAIL(log_block.seal(replay_hint_v, barrier))) { + TRANS_LOG(WARN, "seal log block fail", K(ret)); + } else if (OB_SUCC(ls_tx_ctx_mgr_->get_ls_log_adapter() + ->submit_log(log_block.get_buf(), + log_block.get_size(), + base_scn, + log_cb, + false))) { + busy_cbs_.add_last(log_cb); + } + } + return ret; +} + ERRSIM_POINT_DEF(ERRSIM_DELAY_TX_SUBMIT_LOG); int ObPartTransCtx::submit_log_impl_(const ObTxLogType log_type) @@ -3887,12 +3801,6 @@ int ObPartTransCtx::submit_log_impl_(const ObTxLogType log_type) K(trans_id_), K(ls_id_), K(big_segment_info_)); } else { switch (log_type) { - // for instant logging during execution, - // use non-block clog interface. - case ObTxLogType::TX_REDO_LOG: { - ret = submit_redo_log_(); - break; - } // leader switch case ObTxLogType::TX_ACTIVE_INFO_LOG: { ret = submit_redo_active_info_log_(); @@ -3958,6 +3866,14 @@ int ObPartTransCtx::submit_log_impl_(const ObTxLogType log_type) TRANS_LOG(INFO, "submit_log_impl_ end", KR(ret), K(log_type), K(*this)); #endif } + if (OB_FAIL(ret)) { + handle_submit_log_err_(log_type, ret); + } + return ret; +} + +void ObPartTransCtx::handle_submit_log_err_(const ObTxLogType log_type, int &ret) +{ if (OB_TX_NOLOGCB == ret) { if (REACH_COUNT_PER_SEC(10) && REACH_TIME_INTERVAL(100 * 1000)) { TRANS_LOG(INFO, "can not get log_cb when submit_log", KR(ret), K(log_type), @@ -3993,8 +3909,6 @@ int ObPartTransCtx::submit_log_impl_(const ObTxLogType log_type) } } } - - return ret; } void ObPartTransCtx::reset_redo_lsns_() @@ -4023,12 +3937,15 @@ bool ObPartTransCtx::need_record_log_() const return get_redo_log_no_() >= prev_redo_lsns_count && !sub_state_.is_info_log_submitted(); } +#define bitmap_is_contain(X) (bitmap & (uint64_t)X) + //big row may use a unused log_block to invoke after_submit_log_ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, ObTxLogCb *log_cb, ObRedoLogSubmitHelper *helper) { int ret = OB_SUCCESS; + uint64_t bitmap = 0; const ObTxCbArgArray &cb_arg_array = log_block.get_cb_arg_array(); if (cb_arg_array.count() == 0) { ret = OB_ERR_UNEXPECTED; @@ -4036,11 +3953,15 @@ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, } else if (OB_FAIL(log_cb->get_cb_arg_array().assign(cb_arg_array))) { TRANS_LOG(WARN, "assign cb arg array failed", K(ret)); } else { - if (is_contain(cb_arg_array, ObTxLogType::TX_REDO_LOG) || - is_contain(cb_arg_array, ObTxLogType::TX_ROLLBACK_TO_LOG) || - is_contain(cb_arg_array, ObTxLogType::TX_BIG_SEGMENT_LOG) || - is_contain(cb_arg_array, ObTxLogType::TX_MULTI_DATA_SOURCE_LOG)) { - if (!is_contain(cb_arg_array, ObTxLogType::TX_COMMIT_INFO_LOG)) { + for (int i = 0; i < cb_arg_array.count(); i++) { + bitmap |= (uint64_t)cb_arg_array.at(i).get_log_type(); + } + if (bitmap_is_contain(ObTxLogType::TX_REDO_LOG) || + bitmap_is_contain(ObTxLogType::TX_ROLLBACK_TO_LOG) || + bitmap_is_contain(ObTxLogType::TX_BIG_SEGMENT_LOG) || + bitmap_is_contain(ObTxLogType::TX_MULTI_DATA_SOURCE_LOG)) { + if (!bitmap_is_contain(ObTxLogType::TX_COMMIT_INFO_LOG)) { + TRANS_LOG(TRACE, "redo_lsns.push", K(log_cb->get_lsn())); ret = exec_info_.redo_lsns_.push_back(log_cb->get_lsn()); } } @@ -4048,21 +3969,24 @@ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, if (OB_SUCC(ret) && OB_FAIL(update_rec_log_ts_(false/*for_replay*/, SCN()))) { TRANS_LOG(WARN, "update rec log ts failed", KR(ret), KPC(log_cb), K(*this)); } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_REDO_LOG)) { - log_cb->set_callbacks(helper->callbacks_); - update_max_submitted_seq_no(helper->max_seq_no_); - if (OB_FAIL(mt_ctx_.log_submitted(*helper))) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_REDO_LOG)) { + if (OB_FAIL(log_cb->set_callbacks(helper->callbacks_))) { + ob_abort(); + } + exec_info_.max_submitted_seq_no_.inc_update(helper->max_seq_no_); + helper->log_scn_ = log_cb->get_log_ts(); + if (helper->callback_redo_submitted_ && OB_FAIL(mt_ctx_.log_submitted(*helper))) { TRANS_LOG(ERROR, "fill to do log_submitted on redo log gen", K(ret), K(*this)); } } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_ROLLBACK_TO_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_ROLLBACK_TO_LOG)) { // do nothing } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_MULTI_DATA_SOURCE_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_MULTI_DATA_SOURCE_LOG)) { // do nothing log_cb->get_mds_range().range_submitted(mds_cache_); } - if(OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_BIG_SEGMENT_LOG)) + if(OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_BIG_SEGMENT_LOG)) { add_unsynced_segment_cb_(log_cb); if (big_segment_info_.segment_buf_.is_completed()) { @@ -4070,18 +3994,18 @@ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, big_segment_info_.reuse(); } } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_ACTIVE_INFO_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_ACTIVE_INFO_LOG)) { // do nothing } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_COMMIT_INFO_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_COMMIT_INFO_LOG)) { sub_state_.set_info_log_submitted(); } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_PREPARE_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_PREPARE_LOG)) { sub_state_.set_state_log_submitting(); sub_state_.set_state_log_submitted(); exec_info_.prepare_version_ = SCN::max(log_cb->get_log_ts(), exec_info_.prepare_version_); } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_COMMIT_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_COMMIT_LOG)) { sub_state_.set_state_log_submitting(); sub_state_.set_state_log_submitted(); // elr @@ -4093,11 +4017,11 @@ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, elr_handler_.check_and_early_lock_release(has_row_updated, this); } } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_ABORT_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_ABORT_LOG)) { sub_state_.set_state_log_submitting(); sub_state_.set_state_log_submitted(); } - if (OB_SUCC(ret) && is_contain(cb_arg_array, ObTxLogType::TX_CLEAR_LOG)) { + if (OB_SUCC(ret) && bitmap_is_contain(ObTxLogType::TX_CLEAR_LOG)) { sub_state_.set_state_log_submitting(); sub_state_.set_state_log_submitted(); } @@ -4112,10 +4036,9 @@ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, TRANS_LOG(WARN, "after submit log failed", K(ret), K(trans_id_), K(ls_id_), K(exec_info_), K(*log_cb)); } else { #ifndef NDEBUG - TRANS_LOG(INFO, "after submit log success", K(ret), K(trans_id_), K(ls_id_), K(exec_info_), K(*log_cb)); + TRANS_LOG(INFO, "after submit log success", K(ret), K(trans_id_), K(ls_id_), K(exec_info_), K(*log_cb), KPC(this)); #endif } - REC_TRANS_TRACE_EXT(tlog_, after_submit_log, OB_ID(ret), @@ -4130,9 +4053,7 @@ int ObPartTransCtx::after_submit_log_(ObTxLogBlock &log_block, log_cb->get_lsn()); exec_info_.next_log_entry_no_++; - ObTxLogBlockHeader - block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - log_block.reuse(trans_id_, block_header); + reuse_log_block_(log_block); return ret; } @@ -4156,20 +4077,28 @@ int ObPartTransCtx::get_log_cb_(const bool need_final_cb, ObTxLogCb *&log_cb) log_cb = &final_log_cb_; } } else { - for (int64_t i = 0; OB_SUCC(ret) && i < 2; i++) { - if (!free_cbs_.is_empty()) { + bool empty = false; + { // try fast_path + ObSpinLockGuard guard(log_cb_lock_); + empty = free_cbs_.is_empty(); + if (!empty) { if (OB_ISNULL(log_cb = free_cbs_.remove_first())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "unexpected null log cb", KR(ret), K(*this)); - } else { - break; } - } else { - if (OB_FAIL(extend_log_cbs_())) { + } + } + if (empty) { // slowpath + CtxLockGuard guard; + if (!lock_.is_locked_by_self()) { + get_ctx_guard(guard, CtxLockGuard::MODE::CTX); + } + if (OB_FAIL(extend_log_cbs_(log_cb))) { + if (OB_TX_NOLOGCB != ret) { TRANS_LOG(WARN, "extend log callback failed", K(ret)); - // rewrite ret - ret = OB_TX_NOLOGCB; } + // rewrite ret + ret = OB_TX_NOLOGCB; } } } @@ -4179,20 +4108,35 @@ int ObPartTransCtx::get_log_cb_(const bool need_final_cb, ObTxLogCb *&log_cb) TRANS_LOG(WARN, "unexpected log callback", K(ret)); } else { log_cb->reuse(); - busy_cbs_.add_last(log_cb); } } return ret; } -int ObPartTransCtx::return_log_cb_(ObTxLogCb *log_cb) +int ObPartTransCtx::return_redo_log_cb(ObTxLogCb *log_cb) +{ + int ret = OB_SUCCESS; + if (log_cb == &final_log_cb_) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "redo log cb should not be final_log_cb", K(ret)); + } else { + ret = return_log_cb_(log_cb); + } + return ret; +} + +int ObPartTransCtx::return_log_cb_(ObTxLogCb *log_cb, bool release_final_cb) { int ret = OB_SUCCESS; if (NULL != log_cb) { - busy_cbs_.remove(log_cb); - log_cb->reuse(); if ((&final_log_cb_) != log_cb) { + log_cb->reuse(); + ObSpinLockGuard guard(log_cb_lock_); free_cbs_.add_first(log_cb); + } else if (release_final_cb) { + log_cb->reuse(); + } else { + // for final_log_cb_, it will be used only once, we don't reuse it } } return ret; @@ -4393,7 +4337,7 @@ int ObPartTransCtx::check_replay_avaliable_(const palf::LSN &offset, bool &need_replay) { int ret = OB_SUCCESS; - + need_replay = true; if (IS_NOT_INIT) { TRANS_LOG(WARN, "ObPartTransCtx not inited"); ret = OB_NOT_INIT; @@ -4406,7 +4350,6 @@ int ObPartTransCtx::check_replay_avaliable_(const palf::LSN &offset, // } else if (is_exiting_) { // // ret = OB_TRANS_CTX_NOT_EXIST; } else { - need_replay = true; // check state like ObPartTransCtx::is_trans_valid_for_replay_ if (!exec_info_.max_applying_log_ts_.is_valid()) { // do nothing @@ -4420,18 +4363,20 @@ int ObPartTransCtx::check_replay_avaliable_(const palf::LSN &offset, } } - if (need_replay && !start_replay_ts_.is_valid()) { - start_replay_ts_ = timestamp; - } + if (OB_SUCC(ret)) { + if (need_replay && !start_replay_ts_.is_valid()) { + start_replay_ts_ = timestamp; + } - if (need_replay) { - update_rec_log_ts_(true/*for_replay*/, timestamp); + if (need_replay) { + update_rec_log_ts_(true/*for_replay*/, timestamp); + } } return ret; } -int ObPartTransCtx::push_repalying_log_ts(const SCN log_ts_ns) +int ObPartTransCtx::push_repalying_log_ts(const SCN log_ts_ns, const bool is_first) { int ret = OB_SUCCESS; @@ -4449,11 +4394,15 @@ int ObPartTransCtx::push_repalying_log_ts(const SCN log_ts_ns) exec_info_.max_applying_log_ts_ = log_ts_ns; exec_info_.max_applying_part_log_no_ = 0; } - + if (is_first) { + ctx_tx_data_.set_start_log_ts(log_ts_ns); + } return ret; } -int ObPartTransCtx::push_replayed_log_ts(SCN log_ts_ns, const palf::LSN &offset) +int ObPartTransCtx::push_replayed_log_ts(const SCN log_ts_ns, + const palf::LSN &offset, + const int64_t log_entry_no) { int ret = OB_SUCCESS; @@ -4463,14 +4412,16 @@ int ObPartTransCtx::push_replayed_log_ts(SCN log_ts_ns, const palf::LSN &offset) exec_info_.max_applied_log_ts_ = log_ts_ns; } - if (exec_info_.max_durable_lsn_.is_valid()) { - if (offset > exec_info_.max_durable_lsn_) { - exec_info_.max_durable_lsn_ = offset; - } - } else { + if (!exec_info_.max_durable_lsn_.is_valid() || offset > exec_info_.max_durable_lsn_) { exec_info_.max_durable_lsn_ = offset; } + if (log_entry_no >= exec_info_.next_log_entry_no_) { + // In ActiveInfoLog is replayed, its log_entry_no is the final + // of last leader, set the next_log_entry_no for new leader + exec_info_.next_log_entry_no_ = log_entry_no + 1; + } + if (OB_SUCC(ret)) { if (big_segment_info_.segment_buf_.is_completed() && big_segment_info_.unsynced_segment_part_cbs_.count() > 0) { @@ -4485,6 +4436,7 @@ int ObPartTransCtx::push_replayed_log_ts(SCN log_ts_ns, const palf::LSN &offset) // } } } + return ret; } @@ -4650,12 +4602,29 @@ int ObPartTransCtx::replace_tx_data_with_backup_(const ObTxDataBackup &backup, S return ret; } -void ObPartTransCtx::force_no_need_replay_checksum() +void ObPartTransCtx::force_no_need_replay_checksum(const bool parallel_replay, + const share::SCN &log_ts) { - exec_info_.need_checksum_ = false; + if (ATOMIC_LOAD(&exec_info_.need_checksum_)) { + CtxLockGuard guard(lock_); + force_no_need_replay_checksum_(parallel_replay, log_ts); + } } -void ObPartTransCtx::check_no_need_replay_checksum(const SCN &log_ts) +void ObPartTransCtx::force_no_need_replay_checksum_(const bool parallel_replay, + const share::SCN &log_ts) +{ + if (ATOMIC_LOAD(&exec_info_.need_checksum_)) { + TRANS_LOG(INFO, "set skip calc checksum", K_(trans_id), K_(ls_id), KP(this), K(parallel_replay), K(log_ts)); + if (parallel_replay) { + update_rec_log_ts_(true, log_ts); + } + ATOMIC_STORE(&exec_info_.need_checksum_, false); + mt_ctx_.set_skip_checksum_calc(); + } +} + +void ObPartTransCtx::check_no_need_replay_checksum(const SCN &log_ts, const int index) { // TODO(handora.qc): How to lock the tx_ctx @@ -4663,93 +4632,153 @@ void ObPartTransCtx::check_no_need_replay_checksum(const SCN &log_ts) // log of checksum_scn_(not included). So if the data with this scn is // not replayed with checksum_scn_ <= scn, it means may exist some data // will never be replayed because the memtable will filter the data. - if (exec_info_.checksum_scn_ <= log_ts) { - exec_info_.need_checksum_ = false; - } -} -int ObPartTransCtx::validate_replay_log_entry_no(bool first_created_ctx, - int64_t log_entry_no, - const SCN &log_ts) -{ - int ret = OB_SUCCESS; - - CtxLockGuard guard(lock_); - - if (first_created_ctx || exec_info_.is_empty_ctx_created_by_transfer_) { - if (0 == log_entry_no) { - if (OB_FAIL(ctx_tx_data_.set_start_log_ts(log_ts))) { - TRANS_LOG(WARN, "set start ts failed", K(ret)); - } - } - if (OB_SUCC(ret)) { - exec_info_.next_log_entry_no_ = log_entry_no + 1; - if (exec_info_.next_log_entry_no_ > 1) { - is_incomplete_replay_ctx_ = true; - exec_info_.is_empty_ctx_created_by_transfer_ = false; - exec_info_.need_checksum_ = false; - if (OB_FAIL(supplement_undo_actions_if_exist_())) { - TRANS_LOG(WARN, - "supplement undo actions to a tx data when replaying a transaction from the " - "middle failed.", - K(ret)); + if (ATOMIC_LOAD(&exec_info_.need_checksum_)) { + bool need_skip = true; + bool serial_replay = index == 0; + // the serial replay or the parallel replay in tx-log queue + if (serial_replay) { + const share::SCN serial_final_scn = exec_info_.serial_final_scn_.atomic_load(); + serial_replay = !serial_final_scn.is_valid() || log_ts <= serial_final_scn; + // the log is before serial final point, if either of checksum_scn larger than + // the log_ts, the checksum must contains the log_ts + if (serial_replay) { + CtxLockGuard guard(lock_); // acquire lock for access array + ARRAY_FOREACH_NORET(exec_info_.checksum_scn_, i) { + if (exec_info_.checksum_scn_.at(i) > log_ts) { + need_skip = false; + break; + } } - - TRANS_LOG(INFO, - "This is a incomplete trx which replay from the middle", - K(ret), - K(ls_id_), - K(trans_id_), - K(log_ts), - K(exec_info_.next_log_entry_no_), - K(is_incomplete_replay_ctx_), - K(ctx_tx_data_)); - } else { - exec_info_.is_empty_ctx_created_by_transfer_ = false; } } - } else if (log_entry_no > exec_info_.next_log_entry_no_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, - "unexpected log entry no", - K(ret), - K(trans_id_), - K(ls_id_), - K(log_ts), - K(log_entry_no), - K(exec_info_.next_log_entry_no_)); - } else if (log_entry_no < exec_info_.next_log_entry_no_) { - // do nothing, filtered by max_applying_log_ts_ - exec_info_.is_empty_ctx_created_by_transfer_ = false; - } else { - exec_info_.next_log_entry_no_ = log_entry_no + 1; - exec_info_.is_empty_ctx_created_by_transfer_ = false; + // for parallel replay, check the corresponding list's checksum_scn + if (!serial_replay) { + CtxLockGuard guard(lock_); // acquire lock for access array + if (exec_info_.checksum_scn_.count() <= index) { + // the checksum is not exist + } else if (exec_info_.checksum_scn_.at(index) > log_ts) { + need_skip = false; + } + } + if (need_skip) { + CtxLockGuard guard(lock_); // acquire lock for display ctx + force_no_need_replay_checksum_(index != 0, log_ts); + TRANS_LOG(INFO, "skip checksum, because checksum calc not continous", + K(serial_replay), K(index), K(log_ts), KPC(this)); + } } - - return ret; } +/* + * since 4.3, redo_lsns can not be maintained on follower with order + * because the redo were replay parallelly, instead, redo_lsns only + * maintained on leader and when switch to follower, it will persistent + * redo_lsns with `RecordLog` + */ int ObPartTransCtx::check_and_merge_redo_lsns_(const palf::LSN &offset) { int ret = OB_SUCCESS; - int64_t cnt = exec_info_.redo_lsns_.count(); - // TODO merge without same offset - if (cnt != 0 && exec_info_.redo_lsns_[cnt - 1] == offset) { - TRANS_LOG(INFO, "repeated redo log offset", K(offset), K(exec_info_.redo_lsns_), K(trans_id_)); - } else if (OB_FAIL(exec_info_.redo_lsns_.push_back(offset))) { - TRANS_LOG(WARN, "redo log offset push back error", K(ret), K(offset), K(trans_id_)); + if (!is_support_parallel_replay_()) { + int64_t cnt = exec_info_.redo_lsns_.count(); + // TODO merge without same offset + if (cnt != 0 && exec_info_.redo_lsns_[cnt - 1] == offset) { + TRANS_LOG(INFO, "repeated redo log offset", K(offset), K(exec_info_.redo_lsns_), K(trans_id_)); + } else if (OB_FAIL(exec_info_.redo_lsns_.push_back(offset))) { + TRANS_LOG(WARN, "redo log offset push back error", K(ret), K(offset), K(trans_id_)); + } } return ret; } +/* + * replay redo in tx ctx + * + * since 4.3, support parallel replay redo, and the design principle is + * seperate redo and other logs(named as Txn's Log), redo is belongs to + * memtable (and locktable), and only Txn's Log will replay into Tx ctx + * and affect the Tx ctx's state + * + * for compatible with old version, redo belongs to `tx_log_queue` will + * still handle same as Txn's Log + */ int ObPartTransCtx::replay_redo_in_ctx(const ObTxRedoLog &redo_log, const palf::LSN &offset, const SCN ×tamp, - const int64_t &part_log_no) + const int64_t &part_log_no, + const bool is_tx_log_queue, + const bool serial_final) { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("replay_redo_in_ctx", 10 * 1000); - // const int64_t start = ObTimeUtility::fast_current_time(); + { + CtxLockGuard guard(lock_); + // before 4.3, cluster_version is in RedoLog, and + // the cluster_version_ is initialized to CURRENT_CLUSTER_VERSION when ctx created + // it should be correct with in redo + // since 4.3, cluster_version is in LogBlockHeader, so the cluster_version is correct + // when created, and cluster_version in RedoLog is 0 + if (redo_log.get_cluster_version() > 0) { + ret = correct_cluster_version_(redo_log.get_cluster_version()); + } + + // if we need calc checksum, must don't recycle redo log ts before they were replayed + // otherwise the checksum scn in TxCtx checkpoint will be lag behind recovery scn + // and after the restart, the txn's checksum verify will be skipped + // + // example: + // + // Log sequence of Txn is : 1 -> 2 -> 3 -> 4 + // where 1, 4 in queue 0 (aka tx-log-queue), 2 in queue 2 and 3 in queue 3 + // because of parallel replaying, assume queue 0 replayed 4, queue 2 and 3 not + // replayed 2, 3 yet, then in this moment, a checkpoint are issued, the checksum + // calculate for queue 2 and queue 3 will missing data of log 2 and 3 + // after checkpoint, and 2, 3 replayed, the system will recycle logs 1-4, + // after a restart, recovery from log queue after 4, and 2,3 will not be replayed + // finally the checksum of queue 2,3 not include log sequence 2,3 + // + // the cons of this choice is after restart, the log recycle position + // will be more older, which cause do more times checkpoint of TxCtx + // + if (OB_SUCC(ret) && !is_tx_log_queue && exec_info_.need_checksum_) { + update_rec_log_ts_(true, timestamp); + } + // if this is serial final redo log + // change the logging to parallel logging + if (OB_SUCC(ret) && serial_final) { + if (!is_tx_log_queue) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "serial final redo must be in tx_log_queue", K(ret), KPC(this), K(timestamp)); + usleep(50_ms); + ob_abort(); + } else if (!exec_info_.serial_final_scn_.is_valid()) { + switch_to_parallel_logging_(timestamp); + } + } + } + if (OB_SUCC(ret)) { + ObTransStatistic::get_instance().add_redo_log_replay_count(tenant_id_, 1); + ObTransStatistic::get_instance().add_redo_log_replay_time(tenant_id_, timeguard.get_diff()); + + // for redo compound with Txn Log, go with old ways, this can also handle replay of old version + if (is_tx_log_queue) { + replay_redo_in_ctx_compat_(redo_log, offset, timestamp, part_log_no); + } + } + +#ifndef NDEBUG + TRANS_LOG(INFO, "[Replay Tx] Replay Redo in TxCtx", K(ret), + K(is_tx_log_queue), K(timestamp), K(offset), KPC(this)); +#endif + return ret; +} + +int ObPartTransCtx::replay_redo_in_ctx_compat_(const ObTxRedoLog &redo_log, + const palf::LSN &offset, + const SCN ×tamp, + const int64_t &part_log_no) +{ + int ret = OB_SUCCESS; lib::Worker::CompatMode mode = lib::Worker::CompatMode::INVALID; bool need_replay = true; @@ -4761,69 +4790,112 @@ int ObPartTransCtx::replay_redo_in_ctx(const ObTxRedoLog &redo_log, TRANS_LOG(INFO, "need not replay log", K(redo_log), K(timestamp), K(offset), K(*this)); // no need to replay } else if (OB_FAIL((update_replaying_log_no_(timestamp, part_log_no)))) { - TRANS_LOG(WARN, "update replaying log no failed", K(ret), K(timestamp), K(part_log_no)); + TRANS_LOG(WARN, "update replaying log no failed", K(ret), K(timestamp)); } else { // TODO add clog_encrypt_info_ UNUSED(set_upstream_state(ObTxState::INIT)); - cluster_version_ = redo_log.get_cluster_version(); - // if (0 == redo_log.get_log_no()) { - // if (OB_FAIL(ctx_tx_data_.set_start_log_ts(timestamp))) { - // TRANS_LOG(WARN, "set start ts failed", K(ret)); - // } - // } if (OB_SUCC(ret) && OB_FAIL(check_and_merge_redo_lsns_(offset))) { TRANS_LOG(WARN, "check and merge redo lsns failed", K(ret), K(trans_id_), K(timestamp), K(offset)); } - // const int64_t end = ObTimeUtility::fast_current_time(); - ObTransStatistic::get_instance().add_redo_log_replay_count(tenant_id_, 1); - ObTransStatistic::get_instance().add_redo_log_replay_time(tenant_id_, timeguard.get_diff()); } REC_TRANS_TRACE_EXT(tlog_, replay_redo, OB_ID(ret), ret, - OB_ID(used), timeguard.get_diff(), OB_Y(need_replay), OB_ID(offset), offset.val_, OB_ID(t), timestamp, OB_ID(ref), get_ref()); if (OB_FAIL(ret)) { TRANS_LOG(WARN, "[Replay Tx] Replay Redo in TxCtx Failed", K(ret), K(timestamp), K(offset), K(need_replay), K(redo_log), K(*this)); - } else { -#ifndef NDEBUG - TRANS_LOG(INFO, "[Replay Tx] Replay Redo in TxCtx", K(ret), K(timestamp), K(offset), - K(need_replay), K(redo_log), K(*this)); -#endif } return ret; } +// +// Replay RollbackToLog +// the RollbackToLog operate on memtable, its replay seperate two step +// Step1: add UndoAction to TxData +// Step2: rollback(remove) data on memtable +// +// for Step1, repeatedly replay is handle here +// for Step2, it must be executed even Step1 is should be skipped +// +// When `Branch Savepoint` used, RollbackToLog can be replayed parallelly +// in this situation, Step1 can not be handled efficiently, so repeatedly +// replay such RollbackToLog is possible, maybe use TxData to deduplicate +// is possible. +// +// int ObPartTransCtx::replay_rollback_to(const ObTxRollbackToLog &log, const palf::LSN &offset, const SCN ×tamp, - const int64_t &part_log_no) + const int64_t &part_log_no, + const bool is_tx_log_queue, + const bool pre_barrier) { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("replay_rollback_to", 10 * 1000); // int64_t start = ObTimeUtility::fast_current_time(); CtxLockGuard guard(lock_); bool need_replay = true; - if (OB_FAIL(check_replay_avaliable_(offset, timestamp, part_log_no, need_replay))) { - TRANS_LOG(WARN, "check replay available failed", KR(ret), K(offset), K(timestamp), K(*this)); - } else if (!need_replay) { - TRANS_LOG(INFO, "need not replay log", K(log), K(timestamp), K(offset), K(*this)); - } else if (OB_FAIL((update_replaying_log_no_(timestamp, part_log_no)))) { - TRANS_LOG(WARN, "update replaying log no failed", K(ret), K(timestamp), K(part_log_no)); - } else if (OB_FAIL(rollback_to_savepoint_(log.get_from(), log.get_to()))) { + ObTxSEQ from = log.get_from(); + ObTxSEQ to = log.get_to(); + if (OB_UNLIKELY(from.get_branch() != to.get_branch())) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "invalid savepoint", K(log)); + } + // + // the log is replay in txn log queue + // for parallel replay, a global savepoint after the serial final log + // must set the pre-barrier replay flag + // some branch savepoint also need this, but we can't distinguish + // hence only sanity check for global savepoint + // + else if (is_tx_log_queue) { + if (is_parallel_logging() // has enter parallel logging + && to.get_branch() == 0 // is a global savepoint + && timestamp > exec_info_.serial_final_scn_ // it is after the serial final log + && !pre_barrier) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "missing pre barrier flag for parallel replay", KR(ret), K(*this)); + usleep(5000); + ob_abort(); + } else if (OB_FAIL(check_replay_avaliable_(offset, timestamp, part_log_no, need_replay))) { + TRANS_LOG(WARN, "check replay available failed", KR(ret), K(offset), K(timestamp), K(*this)); + } else if (!need_replay) { + TRANS_LOG(INFO, "need not replay log", K(log), K(timestamp), K(offset), K(*this)); + } else if (OB_FAIL((update_replaying_log_no_(timestamp, part_log_no)))) { + TRANS_LOG(WARN, "update replaying log no failed", K(ret), K(timestamp), K(part_log_no)); + } + } else if (exec_info_.need_checksum_ && !has_replay_serial_final_()) { + ret = OB_EAGAIN; + TRANS_LOG(INFO, "branch savepoint should wait replay serial final because of calc checksum", + K(ret), K(timestamp), KP(this), K(trans_id_), K(ls_id_), K(exec_info_)); + } else if (!ctx_tx_data_.get_start_log_ts().is_valid() && OB_FAIL(ctx_tx_data_.set_start_log_ts(timestamp))) { + // update start_log_ts for branch savepoint, because it may replayed before first log in txn queue + TRANS_LOG(WARN, "set tx data start log ts fail", K(ret), K(timestamp), KPC(this)); + } + + // + // Step1, add Undo into TxData, both for parallel replay and serial replay + // + if (OB_SUCC(ret) && need_replay && OB_FAIL(rollback_to_savepoint_(log.get_from(), log.get_to(), timestamp))) { TRANS_LOG(WARN, "replay savepoint_rollback fail", K(ret), K(log), K(offset), K(timestamp), KPC(this)); - } else if (OB_FAIL(check_and_merge_redo_lsns_(offset))) { + } + + // this is compatible code, since 4.3, redo_lsn not collect during replay + if (OB_SUCC(ret) && OB_FAIL(check_and_merge_redo_lsns_(offset))) { TRANS_LOG(WARN, "check and merge redo lsns failed", K(ret), K(trans_id_), K(timestamp), K(offset)); } - if (OB_SUCC(ret)) { - if (OB_FAIL(mt_ctx_.rollback(log.get_to(), log.get_from()))) { + // + // Step2, remove TxNode(s) + // + if (OB_SUCC(ret) && !need_replay) { + if (OB_FAIL(mt_ctx_.rollback(log.get_to(), log.get_from(), timestamp))) { TRANS_LOG(WARN, "mt ctx rollback fail", K(ret), K(log), KPC(this)); } } - if (OB_FAIL(ret)) { + if (OB_FAIL(ret) && OB_EAGAIN != ret) { TRANS_LOG(WARN, "[Replay Tx] Replay RollbackToLog in TxCtx Failed", K(timestamp), K(offset), K(ret), K(need_replay), K(log), KPC(this)); } else { @@ -4832,19 +4904,23 @@ int ObPartTransCtx::replay_rollback_to(const ObTxRollbackToLog &log, K(need_replay), K(log), KPC(this)); #endif } - REC_TRANS_TRACE_EXT(tlog_, - replay_rollback_to, - OB_ID(ret), - ret, - OB_ID(used), - timeguard.get_diff(), - OB_Y(need_replay), - OB_ID(offset), - offset.val_, - OB_ID(t), - timestamp, - OB_ID(ref), - get_ref()); + + if (OB_EAGAIN != ret) { + REC_TRANS_TRACE_EXT(tlog_, + replay_rollback_to, + OB_ID(ret), + ret, + OB_ID(used), + timeguard.get_diff(), + OB_Y(need_replay), + OB_ID(offset), + offset.val_, + OB_ID(t), + timestamp, + OB_ID(ref), + get_ref()); + } + return ret; } @@ -4870,6 +4946,8 @@ int ObPartTransCtx::replay_active_info(const ObTxActiveInfoLog &log, TRANS_LOG(WARN, "set scheduler error", K(ret), K(log), K(*this)); } else if (OB_FAIL(set_app_trace_id_(log.get_app_trace_id()))) { TRANS_LOG(WARN, "set app trace id error", K(ret), K(log), K(*this)); + } else if (log.get_cluster_version() > 0 && + OB_FAIL(correct_cluster_version_(log.get_cluster_version()))) { } else { exec_info_.trans_type_ = log.get_trans_type(); if (log.is_dup_tx()) { @@ -4886,8 +4964,9 @@ int ObPartTransCtx::replay_active_info(const ObTxActiveInfoLog &log, last_op_sn_ = log.get_last_op_sn(); first_scn_ = log.get_first_seq_no(); last_scn_ = log.get_last_seq_no(); - cluster_version_ = log.get_cluster_version(); - update_max_submitted_seq_no(log.get_max_submitted_seq_no()); + + exec_info_.max_submitted_seq_no_.inc_update(log.get_max_submitted_seq_no()); + exec_info_.serial_final_seq_no_ = log.get_serial_final_seq_no(); exec_info_.data_complete_ = true; } if (OB_FAIL(ret)) { @@ -4936,15 +5015,20 @@ int ObPartTransCtx::assign_commit_parts(const share::ObLSArray &log_participants int ObPartTransCtx::replay_commit_info(const ObTxCommitInfoLog &commit_info_log, const palf::LSN &offset, const SCN ×tamp, - const int64_t &part_log_no) + const int64_t &part_log_no, + const bool pre_barrier) { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("replay_commit_info", 10 * 1000); // const int64_t start = ObTimeUtility::fast_current_time(); bool need_replay = true; CtxLockGuard guard(lock_); - - if (OB_FAIL(check_replay_avaliable_(offset, timestamp, part_log_no, need_replay))) { + if (is_parallel_logging() && !pre_barrier) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "missing pre barrier flag for parallel replay", KR(ret), K(*this)); + usleep(5000); + ob_abort(); + } else if (OB_FAIL(check_replay_avaliable_(offset, timestamp, part_log_no, need_replay))) { TRANS_LOG(WARN, "check replay available failed", KR(ret), K(offset), K(timestamp), K(*this)); } else if (!need_replay) { TRANS_LOG(INFO, "need not replay log", K(commit_info_log), K(timestamp), K(offset), K(*this)); @@ -4986,7 +5070,9 @@ int ObPartTransCtx::replay_commit_info(const ObTxCommitInfoLog &commit_info_log, } can_elr_ = commit_info_log.is_elr(); - cluster_version_ = commit_info_log.get_cluster_version(); + if (commit_info_log.get_cluster_version() > 0) { + ret = correct_cluster_version_(commit_info_log.get_cluster_version()); + } sub_state_.set_info_log_submitted(); epoch_ = commit_info_log.get_epoch(); reset_redo_lsns_(); @@ -4996,7 +5082,7 @@ int ObPartTransCtx::replay_commit_info(const ObTxCommitInfoLog &commit_info_log, } else if (is_local_tx_()) { set_durable_state_(ObTxState::REDO_COMPLETE); set_upstream_state(ObTxState::REDO_COMPLETE); - } else if (is_incomplete_replay_ctx_) { + } else if (replay_completeness_.is_incomplete()) { set_durable_state_(ObTxState::REDO_COMPLETE); set_upstream_state(ObTxState::REDO_COMPLETE); // incomplete replay ctx will exiting by replay commit/abort/clear, no need to depend on 2PC @@ -5081,7 +5167,7 @@ int ObPartTransCtx::replay_prepare(const ObTxPrepareLog &prepare_log, exec_info_.trans_type_ = TransType::DIST_TRANS; mt_ctx_.set_prepare_version(timestamp); ObTwoPhaseCommitLogType two_phase_log_type = ObTwoPhaseCommitLogType::OB_LOG_TX_MAX; - if (is_incomplete_replay_ctx_) { + if (replay_completeness_.is_incomplete()) { set_durable_state_(ObTxState::PREPARE); set_upstream_state(ObTxState::PREPARE); // incomplete replay ctx will exiting by replay commit/abort/clear, no need to depend on 2PC @@ -5132,13 +5218,11 @@ int ObPartTransCtx::replay_commit(const ObTxCommitLog &commit_log, { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("replay_commit", 10 * 1000); - + CtxLockGuard guard(lock_); // const int64_t start = ObTimeUtility::fast_current_time(); const SCN commit_version = commit_log.get_commit_version(); bool need_replay = true; - CtxLockGuard guard(lock_); - // TODO replace participants_ with prepare_log_info_arr_ for transfer if (OB_FAIL(check_replay_avaliable_(offset, timestamp, part_log_no, need_replay))) { TRANS_LOG(WARN, "check replay available failed", KR(ret), K(offset), K(timestamp), K(*this)); @@ -5161,7 +5245,7 @@ int ObPartTransCtx::replay_commit(const ObTxCommitLog &commit_log, } else { if (OB_SUCC(ret)) { if ((!commit_log.get_multi_source_data().empty() || !exec_info_.multi_data_source_.empty()) - && is_incomplete_replay_ctx_) { + && replay_completeness_.is_incomplete()) { // ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "mds part_ctx can not replay from the middle", K(ret), K(timestamp), K(offset), K(commit_log), KPC(this)); @@ -5184,7 +5268,7 @@ int ObPartTransCtx::replay_commit(const ObTxCommitLog &commit_log, set_upstream_state(ObTxState::COMMIT); } else { ObTwoPhaseCommitLogType two_phase_log_type = ObTwoPhaseCommitLogType::OB_LOG_TX_MAX; - if (is_incomplete_replay_ctx_) { + if (replay_completeness_.is_incomplete()) { set_durable_state_(ObTxState::COMMIT); set_upstream_state(ObTxState::COMMIT); // incomplete replay ctx will exiting by replay commit/abort/clear, no need to depend on @@ -5206,7 +5290,7 @@ int ObPartTransCtx::replay_commit(const ObTxCommitLog &commit_log, if (OB_SUCC(ret)) { const uint64_t checksum = - (exec_info_.need_checksum_ && !is_incomplete_replay_ctx_ ? commit_log.get_checksum() : 0); + (exec_info_.need_checksum_ && replay_completeness_.is_complete() ? commit_log.get_checksum() : 0); mt_ctx_.set_replay_compact_version(replay_compact_version); if (OB_FAIL(notify_table_lock_(timestamp, @@ -5223,7 +5307,11 @@ int ObPartTransCtx::replay_commit(const ObTxCommitLog &commit_log, timestamp, cluster_version_, checksum))) { - TRANS_LOG(WARN, "trans replay commit failed", KR(ret), "context", *this); + TRANS_LOG(WARN, "trans replay commit failed", KR(ret), K(commit_log), KPC(this)); + if (OB_CHECKSUM_ERROR == ret) { + usleep(500000); + ob_abort(); + } } else if ((!ctx_tx_data_.is_read_only()) && OB_FAIL(ctx_tx_data_.insert_into_tx_table())) { TRANS_LOG(WARN, "insert to tx table failed", KR(ret), K(*this)); } else if (is_local_tx_()) { @@ -5253,6 +5341,7 @@ int ObPartTransCtx::replay_commit(const ObTxCommitLog &commit_log, K(commit_log), K(*this)); #endif } + return ret; } @@ -5263,10 +5352,9 @@ int ObPartTransCtx::replay_clear(const ObTxClearLog &clear_log, { int ret = OB_SUCCESS; common::ObTimeGuard timeguard("replay_clear", 10 * 1000); + CtxLockGuard guard(lock_); // const int64_t start = ObTimeUtility::fast_current_time(); bool need_replay = true; - CtxLockGuard guard(lock_); - if (OB_FAIL(check_replay_avaliable_(offset, timestamp, part_log_no, need_replay))) { TRANS_LOG(WARN, "check replay available failed", KR(ret), K(offset), K(timestamp), K(*this)); } else if (!need_replay) { @@ -5290,7 +5378,7 @@ int ObPartTransCtx::replay_clear(const ObTxClearLog &clear_log, // ret = OB_ERR_UNEXPECTED; } else { ObTwoPhaseCommitLogType two_phase_log_type = ObTwoPhaseCommitLogType::OB_LOG_TX_MAX; - if (is_incomplete_replay_ctx_) { + if (replay_completeness_.is_incomplete()) { set_durable_state_(ObTxState::CLEAR); set_upstream_state(ObTxState::CLEAR); // incomplete replay ctx will exiting by replay commit/abort/clear, no need to depend on 2PC @@ -5357,7 +5445,7 @@ int ObPartTransCtx::replay_abort(const ObTxAbortLog &abort_log, } else { if (OB_SUCC(ret)) { if ((!abort_log.get_multi_source_data().empty() || !exec_info_.multi_data_source_.empty()) - && is_incomplete_replay_ctx_) { + && replay_completeness_.is_incomplete()) { // ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "mds part_ctx can not replay from the middle", K(ret), K(timestamp), K(offset), K(abort_log), KPC(this)); @@ -5380,7 +5468,7 @@ int ObPartTransCtx::replay_abort(const ObTxAbortLog &abort_log, set_upstream_state(ObTxState::ABORT); } else { ObTwoPhaseCommitLogType two_phase_log_type = ObTwoPhaseCommitLogType::OB_LOG_TX_MAX; - if (is_incomplete_replay_ctx_) { + if (replay_completeness_.is_incomplete()) { set_durable_state_(ObTxState::ABORT); set_upstream_state(ObTxState::ABORT); // incomplete replay ctx will exiting by replay commit/abort/clear, no need to depend on 2PC @@ -5467,7 +5555,7 @@ int ObPartTransCtx::replay_multi_data_source(const ObTxMultiDataSourceLog &log, } if (OB_SUCC(ret)) { - if (is_incomplete_replay_ctx_) { + if (replay_completeness_.is_incomplete()) { // ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "mds part_ctx can not replay from the middle", K(ret), K(timestamp), K(lsn), K(log), KPC(this)); @@ -5776,7 +5864,7 @@ int ObPartTransCtx::switch_to_follower_forcedly(ObIArray &cb // do nothing } else if (OB_FALSE_IT(mt_ctx_.merge_multi_callback_lists_for_changing_leader())) { // do nothing - } else if (!mt_ctx_.is_all_redo_submitted() && OB_FAIL(mt_ctx_.clean_unlog_callbacks())) { + } else if (OB_FAIL(mt_ctx_.clean_unlog_callbacks())) { TRANS_LOG(WARN, "clear unlog callbacks", KR(ret), K(*this)); } @@ -6034,21 +6122,16 @@ int ObPartTransCtx::update_rec_log_ts_(bool for_replay, const SCN &rec_log_ts) share::SCN min_big_segment_rec_scn = get_min_unsyncd_segment_scn_(); // The semantic of the rec_log_ts means the log ts of the first state change - // after previous checkpoint. So it is only updated if it is unset. - if (!rec_log_ts_.is_valid()) { - if (for_replay) { - if (!rec_log_ts.is_valid()) { - TRANS_LOG(WARN, "update rec log ts failed", K(*this), K(rec_log_ts)); - ret = OB_ERR_UNEXPECTED; - } else { - // Case 1: As follower, the replay is in order, so we can simply set it as - // the log ts of the latest replayed log. because all state changes in - // logs before max_durable_log_ts all successfully contained in this - // checkpoint and no new state changes after max_durable_log_ts is - // contained in the checkpoint. - rec_log_ts_ = rec_log_ts; - } - } else { + // after previous checkpoint. + if (for_replay) { + // follower may support parallel replay redo, so must do dec update + if (!rec_log_ts_.is_valid()) { + rec_log_ts_ = rec_log_ts; + } else if (rec_log_ts_ > rec_log_ts){ + rec_log_ts_ = rec_log_ts; + } + } else { + if (!rec_log_ts_.is_valid()) { // Case 2: As leader, the application is discrete and not in order, so we // should set it as the log ts of the first log submmitted during // continuous logging(we call it FCL later) because all log of the txn with @@ -6144,12 +6227,17 @@ int ObPartTransCtx::refresh_rec_log_ts_() int ObPartTransCtx::get_tx_ctx_table_info_(ObTxCtxTableInfo &info) { int ret = OB_SUCCESS; - exec_info_.exec_epoch_ = epoch_; + // leave target_scn to MAX and the callee will choose the greatest + // calculable scn, especially when parallel replay, the max scn of + // a parallel replayed callback-list will be carefully choosen to + // ensure checksum calculation was continous + share::SCN target_scn = share::SCN::max_scn(); if (OB_FAIL(ctx_tx_data_.get_tx_data(info.tx_data_guard_))) { TRANS_LOG(WARN, "get tx data failed", K(ret)); - } else if (OB_FAIL(mt_ctx_.calc_checksum_before_scn( - exec_info_.max_applied_log_ts_, exec_info_.checksum_, exec_info_.checksum_scn_))) { + } else if (exec_info_.need_checksum_ && + OB_FAIL(mt_ctx_.calc_checksum_before_scn(target_scn, + exec_info_.checksum_, exec_info_.checksum_scn_))) { TRANS_LOG(ERROR, "calc checksum before log ts failed", K(ret), KPC(this)); } else if (OB_FAIL(exec_info_.generate_mds_buffer_ctx_array())) { TRANS_LOG(WARN, "fail to generate mds buffer ctx array", K(ret), KPC(this)); @@ -6450,6 +6538,26 @@ int ObPartTransCtx::decide_state_log_barrier_type_( } } + // decide barrier for parallel logging + if (OB_SUCC(ret)) { + if (((state_log_type == ObTxLogType::TX_COMMIT_INFO_LOG) + || (state_log_type == ObTxLogType::TX_ABORT_LOG)) + && OB_UNLIKELY(is_parallel_logging())) { + using LogBarrierType = logservice::ObReplayBarrierType; + switch(final_barrier_type) { + case LogBarrierType::STRICT_BARRIER: + case LogBarrierType::PRE_BARRIER: + break; + case LogBarrierType::NO_NEED_BARRIER: + final_barrier_type = LogBarrierType::PRE_BARRIER; + break; + default: + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "unexpected barrier type", K(final_barrier_type)); + } + } + } + if (OB_SUCC(ret) && final_barrier_type != logservice::ObReplayBarrierType::NO_NEED_BARRIER && final_barrier_type != logservice::ObReplayBarrierType::INVALID_BARRIER) { TRANS_LOG(INFO, "decide a valid barrier type for state_log", K(ret), @@ -6481,10 +6589,7 @@ int ObPartTransCtx::submit_multi_data_source_() { int ret = OB_SUCCESS; ObTxLogBlock log_block; - const int64_t replay_hint = static_cast(trans_id_.get_id()); - ObTxLogBlockHeader log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, - exec_info_.scheduler_); - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block failed", KR(ret), K(*this)); } else { ret = submit_multi_data_source_(log_block); @@ -6498,7 +6603,7 @@ int ObPartTransCtx::submit_multi_data_source_(ObTxLogBlock &log_block) logservice::ObReplayBarrierType barrier_type = logservice::ObReplayBarrierType::NO_NEED_BARRIER; share::SCN mds_base_scn; - + const int64_t replay_hint = trans_id_.get_id(); ObTxLogCb *log_cb = nullptr; if (mds_cache_.count() > 0) { ObTxMultiDataSourceLog log; @@ -6542,22 +6647,16 @@ int ObPartTransCtx::submit_multi_data_source_(ObTxLogBlock &log_block) TRANS_LOG(INFO, "construct big multi data source",K(ret),K(trans_id_),K(ls_id_),K(log)); } } - } else if (barrier_type != logservice::ObReplayBarrierType::NO_NEED_BARRIER - && OB_FAIL( - log_block.rewrite_barrier_log_block(trans_id_.get_id(), barrier_type))) { - TRANS_LOG(WARN, "rewrite multi data source log barrier failed", K(ret)); - return_log_cb_(log_cb); - log_cb = NULL; } else if (log_block.get_cb_arg_array().count() == 0) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); log_cb = nullptr; + } else if ((mds_base_scn.is_valid() ? OB_FALSE_IT(mds_base_scn = share::SCN::scn_inc(mds_base_scn)) : OB_FALSE_IT(mds_base_scn.set_min()))) { // do nothing - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), mds_base_scn, log_cb, false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, mds_base_scn, log_cb, replay_hint, barrier_type))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); release_ctx_ref_(); } else if (OB_FAIL(after_submit_log_(log_block, log_cb, NULL))) { @@ -6716,7 +6815,7 @@ int ObPartTransCtx::notify_data_source_(const NotifyType notify_type, arg.for_replay_ = for_replay; arg.notify_type_ = notify_type; arg.is_force_kill_ = is_force_kill; - arg.is_incomplete_replay_ = is_incomplete_replay_ctx_; + arg.is_incomplete_replay_ = replay_completeness_.is_incomplete(); int64_t total_time = 0; @@ -6999,7 +7098,8 @@ int ObPartTransCtx::dup_table_tx_redo_sync_() } int ObPartTransCtx::submit_pending_log_block_(ObTxLogBlock &log_block, - memtable::ObRedoLogSubmitHelper &helper) + memtable::ObRedoLogSubmitHelper &helper, + const logservice::ObReplayBarrierType &barrier) { int ret = OB_SUCCESS; @@ -7012,7 +7112,7 @@ int ObPartTransCtx::submit_pending_log_block_(ObTxLogBlock &log_block, || is_contain(log_block.get_cb_arg_array(), ObTxLogType::TX_COMMIT_LOG)) { need_final_cb = true; } - + const int64_t replay_hint = trans_id_.get_id(); ObTxLogCb *log_cb = NULL; if (OB_FAIL(prepare_log_cb_(need_final_cb, log_cb))) { if (OB_UNLIKELY(OB_TX_NOLOGCB != ret)) { @@ -7023,11 +7123,13 @@ int ObPartTransCtx::submit_pending_log_block_(ObTxLogBlock &log_block, TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; + } else if (log_cb->reserve_callbacks(helper.callbacks_.count())) { + TRANS_LOG(WARN, "resolve callbacks failed", K(ret), KPC(this)); + return_log_cb_(log_cb); + log_cb = NULL; } else if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), share::SCN::min_scn(), log_cb, - false, log_block.get_cb_arg_array()))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, share::SCN::min_scn(), log_cb, replay_hint, barrier))) { TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); return_log_cb_(log_cb); log_cb = NULL; @@ -7057,7 +7159,7 @@ int ObPartTransCtx::check_dup_trx_with_submitting_all_redo(ObTxLogBlock &log_blo if (ls_tx_ctx_mgr_->get_ls_log_adapter()->has_dup_tablet()) { if (!sub_state_.is_info_log_submitted() && get_downstream_state() < ObTxState::REDO_COMPLETE) { if (exec_info_.redo_lsns_.count() > 0 || exec_info_.prev_record_lsn_.is_valid() - || !helper.callbacks_.is_empty()) { + || helper.callbacks_.count() > 0) { set_dup_table_tx_(); } @@ -7395,7 +7497,7 @@ int ObPartTransCtx::supplement_undo_actions_if_exist_() TRANS_LOG(ERROR, "get tx data from ctx tx data failed", KR(ret)); } else if (OB_NOT_NULL(guard.tx_data()->undo_status_list_.head_)) { ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "invalid ctx tx data", KR(ret), KPC(tx_data)); + TRANS_LOG(ERROR, "invalid ctx tx data", KR(ret), KPC(guard.tx_data())); } else if (OB_FAIL(ctx_tx_data_.deep_copy_tx_data_out(tmp_tx_data_guard))) { TRANS_LOG(WARN, "deep copy tx data in ctx tx data failed.", KR(ret), K(ctx_tx_data_), KPC(this)); @@ -7405,8 +7507,7 @@ int ObPartTransCtx::supplement_undo_actions_if_exist_() WARN, "supplement undo actions to a tx data when replaying a transaction from the middle failed.", KR(ret), K(ctx_tx_data_), KPC(this)); - } else if (OB_FAIL( - ctx_tx_data_.replace_tx_data(tmp_tx_data_guard.tx_data()))) { + } else if (OB_FAIL(ctx_tx_data_.recover_tx_data(tmp_tx_data_guard.tx_data()))) { TRANS_LOG(WARN, "replace tx data in ctx tx data failed.", KR(ret), K(ctx_tx_data_), KPC(this)); } @@ -7415,7 +7516,7 @@ int ObPartTransCtx::supplement_undo_actions_if_exist_() int ObPartTransCtx::check_status() { - CtxLockGuard guard(lock_); + CtxLockGuard guard(lock_, CtxLockGuard::MODE::ACCESS); return check_status_(); } @@ -7435,7 +7536,7 @@ int ObPartTransCtx::check_status() * in order to reuse this routine, do `check txn is active` at first, thus even if * an active txn which has switched into follower also pass these check */ -int ObPartTransCtx::check_status_() +inline int ObPartTransCtx::check_status_() { int ret = OB_SUCCESS; if (OB_UNLIKELY(is_committing_())) { @@ -7480,40 +7581,63 @@ int ObPartTransCtx::check_status_() * purpose: * 1) verify transaction ctx is *writable* * 2) acquire memtable ctx's ref - * 3) alloc data_scn + * 3) alloc data_scn if not specified */ -int ObPartTransCtx::start_access(const ObTxDesc &tx_desc, ObTxSEQ &data_scn) +int ObPartTransCtx::start_access(const ObTxDesc &tx_desc, ObTxSEQ &data_scn, const int16_t branch) { int ret = OB_SUCCESS; - CtxLockGuard guard(lock_); - if (OB_FAIL(check_status_())) { - } else if (tx_desc.op_sn_ < last_op_sn_) { - ret = OB_TRANS_SQL_SEQUENCE_ILLEGAL; - TRANS_LOG(WARN, "stale access operation", K(ret), - K_(tx_desc.op_sn), K_(last_op_sn), KPC(this), K(tx_desc)); - } else { - if (!data_scn.is_valid()) { - data_scn = tx_desc.inc_and_get_tx_seq(0); - } - ++pending_write_; - last_scn_ = MAX(data_scn, last_scn_); - if (!first_scn_.is_valid()) { - first_scn_ = last_scn_; - } - if (tx_desc.op_sn_ != last_op_sn_) { - last_op_sn_ = tx_desc.op_sn_; + int pending_write = -1; + const bool alloc = !data_scn.is_valid(); + int callback_list_idx = 0; + { + CtxLockGuard guard(lock_, CtxLockGuard::MODE::ACCESS); + if (OB_FAIL(check_status_())) { + } else if (tx_desc.op_sn_ < last_op_sn_) { + ret = OB_TRANS_SQL_SEQUENCE_ILLEGAL; + TRANS_LOG(WARN, "stale access operation", K(ret), + K_(tx_desc.op_sn), K_(last_op_sn), KPC(this), K(tx_desc)); + } else { + if (tx_desc.op_sn_ != last_op_sn_) { + last_op_sn_ = tx_desc.op_sn_; + } + if (alloc) { + data_scn = tx_desc.inc_and_get_tx_seq(branch); + } + last_scn_ = MAX(data_scn, last_scn_); + if (!first_scn_.is_valid()) { + first_scn_ = last_scn_; + } + pending_write = ATOMIC_AAF(&pending_write_, 1); + // others must wait the first thread of parallel open the write epoch + // hence this must be done in lock + if (data_scn.support_branch() && pending_write == 1) { + callback_list_idx = mt_ctx_.acquire_callback_list(true, false); + } } + } + // other operations are allowed to out of lock + if (OB_SUCC(ret)) { mt_ctx_.inc_ref(); - mt_ctx_.acquire_callback_list(); + if (data_scn.support_branch()) { // NEW version >= 4.3 + if (pending_write != 1) { + callback_list_idx = mt_ctx_.acquire_callback_list(false, false); + } + // remember selected callback_list idx into seq_no + if (data_scn.get_branch() == 0 && alloc && callback_list_idx != 0) { + data_scn.set_branch(callback_list_idx); + } + } else { // OLD version < 4.3 + mt_ctx_.acquire_callback_list(false, true /* need merge to main */); + } } last_request_ts_ = ObClockGenerator::getClock(); - TRANS_LOG(TRACE, "start_access", K(ret), KPC(this)); + TRANS_LOG(TRACE, "start_access", K(ret), K(data_scn.support_branch()), K(data_scn), KPC(this)); REC_TRANS_TRACE_EXT(tlog_, start_access, OB_ID(ret), ret, - OB_ID(trace_id), ObCurTraceId::get_trace_id_str(), + OB_ID(trace_id), *ObCurTraceId::get(), OB_ID(opid), tx_desc.op_sn_, OB_ID(data_seq), data_scn.cast_to_int(), - OB_ID(pending), pending_write_, + OB_ID(pending), pending_write, OB_ID(ref), get_ref(), OB_ID(tid), get_itid() + 1); return ret; @@ -7530,14 +7654,14 @@ int ObPartTransCtx::start_access(const ObTxDesc &tx_desc, ObTxSEQ &data_scn) int ObPartTransCtx::end_access() { int ret = OB_SUCCESS; - CtxLockGuard guard(lock_); - --pending_write_; + // to reduce lock contention, these operation is out of lock + int pending_write = ATOMIC_SAF(&pending_write_, 1); + TRANS_LOG(TRACE, "end_access", K(ret), KPC(this)); mt_ctx_.dec_ref(); mt_ctx_.revert_callback_list(); - TRANS_LOG(TRACE, "end_access", K(ret), KPC(this)); REC_TRANS_TRACE_EXT(tlog_, end_access, OB_ID(opid), last_op_sn_, - OB_ID(pending), pending_write_, + OB_ID(pending), pending_write, OB_ID(ref), get_ref(), OB_ID(tid), get_itid() + 1); return ret; @@ -7546,10 +7670,10 @@ int ObPartTransCtx::end_access() /* * rollback_to_savepoint - rollback to savepoint * - * @op_sn - operation sequence number, used to reject out of order msg - * @from_scn - the start position of rollback, inclusive - * @to_scn - the end position of rollback, exclusive - * + * @op_sn - operation sequence number, used to reject out of order msg + * @from_scn - the start position of rollback, inclusive + * @to_scn - the end position of rollback, exclusive + * * savepoint may be created in these ways: * 1) created at txn scheduler, named Global-Savepoint * 2) created at txn participant server, named Local-Savepoint @@ -7593,16 +7717,18 @@ int ObPartTransCtx::rollback_to_savepoint(const int64_t op_sn, } else if (op_sn < last_op_sn_) { ret = OB_TRANS_SQL_SEQUENCE_ILLEGAL; } else if (FALSE_IT(last_op_sn_ = op_sn)) { - } else if (pending_write_ > 0) { + } else if ((to_scn.get_branch() == 0) && pending_write_ > 0) { + // for branch savepoint rollback, pending_write !=0 almostly ret = OB_NEED_RETRY; TRANS_LOG(WARN, "has pending write, rollback blocked", K(ret), K(pending_write_), KPC(this)); } else if (last_scn_ <= to_scn) { - TRANS_LOG(INFO, "rollback succeed trivially", K(trans_id_), K(ls_id_), K(op_sn), K(to_scn), K_(last_scn)); - } else if (OB_FAIL(rollback_to_savepoint_(from_scn, to_scn))) { + TRANS_LOG(INFO, "rollback succeed trivially", K_(trans_id), + K_(ls_id), K(op_sn), K(to_scn), K_(last_scn)); + } else if (OB_FAIL(rollback_to_savepoint_(from_scn, to_scn, share::SCN::invalid_scn()))) { TRANS_LOG(WARN, "rollback_to_savepoint fail", K(ret), K(from_scn), K(to_scn), K(op_sn), KPC(this)); - } else { + } else if (to_scn.get_branch() == 0){ last_scn_ = to_scn; } // must add downstream parts when return success @@ -7627,7 +7753,8 @@ int ObPartTransCtx::rollback_to_savepoint(const int64_t op_sn, } int ObPartTransCtx::rollback_to_savepoint_(const ObTxSEQ from_scn, - const ObTxSEQ to_scn) + const ObTxSEQ to_scn, + const share::SCN replay_scn) { int ret = OB_SUCCESS; @@ -7645,11 +7772,16 @@ int ObPartTransCtx::rollback_to_savepoint_(const ObTxSEQ from_scn, tmp_tx_data_guard.reset(); if (is_follower_()) { /* Follower */ ObUndoAction undo_action(from_scn, to_scn); - if (OB_FAIL(ctx_tx_data_.add_undo_action(undo_action))) { + // _NOTICE_ must load Undo(s) from TxDataTable before overwriten + if (replay_completeness_.is_unknown() && + !ctx_tx_data_.has_recovered_from_tx_table() && + OB_FAIL(supplement_undo_actions_if_exist_())) { + TRANS_LOG(WARN, "load undos from tx table fail", K(ret), KPC(this)); + } else if (OB_FAIL(ctx_tx_data_.add_undo_action(undo_action))) { TRANS_LOG(WARN, "recrod undo info fail", K(ret), K(from_scn), K(to_scn), KPC(this)); } else if (OB_FAIL(ctx_tx_data_.deep_copy_tx_data_out(tmp_tx_data_guard))) { TRANS_LOG(WARN, "deep copy tx data failed", KR(ret), K(*this)); - } else if (FALSE_IT(tmp_tx_data_guard.tx_data()->end_scn_ = exec_info_.max_applying_log_ts_)) { + } else if (FALSE_IT(tmp_tx_data_guard.tx_data()->end_scn_ = share::SCN::max(replay_scn, exec_info_.max_applying_log_ts_))) { } else if (OB_FAIL(ctx_tx_data_.insert_tmp_tx_data(tmp_tx_data_guard.tx_data()))) { TRANS_LOG(WARN, "insert to tx table failed", KR(ret), K(*this)); } @@ -7673,7 +7805,7 @@ int ObPartTransCtx::rollback_to_savepoint_(const ObTxSEQ from_scn, // step 2: remove TxNode(s) from memtable if (OB_SUCC(ret)) { - if (OB_FAIL(mt_ctx_.rollback(to_scn, from_scn))) { + if (OB_FAIL(mt_ctx_.rollback(to_scn, from_scn, replay_scn))) { TRANS_LOG(WARN, "undo provisinal modifies fail", K(ret), K(from_scn), K(to_scn), KPC(this)); } @@ -7688,14 +7820,21 @@ int ObPartTransCtx::submit_rollback_to_log_(const ObTxSEQ from_scn, { int ret = OB_SUCCESS; ObTxLogBlock log_block; - const auto replay_hint = static_cast(trans_id_.get_id()); ObTxRollbackToLog log(from_scn, to_scn); ObTxLogCb *log_cb = NULL; - - ObTxLogBlockHeader - log_block_header(cluster_id_, exec_info_.next_log_entry_no_, trans_id_, exec_info_.scheduler_); - - if (OB_FAIL(log_block.init(replay_hint, log_block_header))) { + int64_t replay_hint = trans_id_.get_id(); + logservice::ObReplayBarrierType barrier = logservice::ObReplayBarrierType::NO_NEED_BARRIER; + if (to_scn.support_branch() && is_parallel_logging()) { + const int16_t branch_id = to_scn.get_branch(); + if (branch_id != 0 && to_scn.get_seq() > exec_info_.serial_final_seq_no_.get_seq()) { + replay_hint += mt_ctx_.get_tx_seq_replay_idx(to_scn); + } else { + // either this is a global savepoint or the savepoint is before serial final point + // must wait the redo log after this savepoint replayed + barrier = logservice::ObReplayBarrierType::PRE_BARRIER; + } + } + if (OB_FAIL(init_log_block_(log_block))) { TRANS_LOG(WARN, "init log block fail", K(ret), KPC(this)); } else if (OB_FAIL(exec_info_.redo_lsns_.reserve(exec_info_.redo_lsns_.count() + 1))) { TRANS_LOG(WARN, "reserve memory for redo lsn failed", K(ret)); @@ -7708,9 +7847,7 @@ int ObPartTransCtx::submit_rollback_to_log_(const ObTxSEQ from_scn, TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(log_block)); return_log_cb_(log_cb); log_cb = NULL; - } else if (OB_FAIL(submit_log_if_allow( - log_block.get_buf(), log_block.get_size(), SCN::min_scn(), log_cb, false, log_block.get_cb_arg_array() - ))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, SCN::min_scn(), log_cb, replay_hint, barrier))) { TRANS_LOG(WARN, "submit log fail", K(ret), K(log_block), KPC(this)); return_log_cb_(log_cb); } else if (OB_FAIL(acquire_ctx_ref())) { @@ -7863,7 +8000,7 @@ int ObPartTransCtx::insert_into_retain_ctx_mgr_(RetainCause cause, if (OB_FAIL(ret)) { // do nothing - } else if (OB_FAIL(mds_retain_func_ptr->init(this, cause, log_ts, lsn))) { + } else if (OB_FAIL(mds_retain_func_ptr->init(this, cause, log_ts))) { TRANS_LOG(WARN, "init retain ctx functor failed", K(ret), KPC(this)); } else if (OB_FAIL(retain_ctx_mgr.push_retain_ctx(retain_ctx_func_ptr_, retain_lock_timeout))) { TRANS_LOG(WARN, "push into retain_ctx_mgr failed", K(ret), KPC(this)); @@ -7912,7 +8049,7 @@ int ObPartTransCtx::do_local_tx_end_(TxEndAction tx_end_action) } case TxEndAction::ABORT_TX: { ret = do_local_abort_tx_(); - if (OB_SUCC(ret)) { + if (OB_SUCC(ret) && part_trans_action_ != ObPartTransAction::COMMIT) { part_trans_action_ = ObPartTransAction::ABORT; } break; @@ -7923,6 +8060,11 @@ int ObPartTransCtx::do_local_tx_end_(TxEndAction tx_end_action) } case TxEndAction::DELAY_ABORT_TX: { sub_state_.set_force_abort(); + // NOTE: clean unlog callbacks is requried: + // if mvcc-row is too large and can't be serialized, freeze thread + // will delay_abort the txn, if don't clean unlog_callbacks + // the memtable's freeze will be blocked + ret = mt_ctx_.clean_unlog_callbacks(); break; } default: { @@ -8788,34 +8930,6 @@ int ObPartTransCtx::do_transfer_out_tx_op(const SCN data_end_scn, return ret; } -int ObPartTransCtx::submit_log_if_allow(const char *buf, - const int64_t size, - const share::SCN &base_ts, - ObTxBaseLogCb *cb, - const bool need_nonblock, - const ObTxCbArgArray &cb_arg_array) -{ - int ret = OB_SUCCESS; - bool is_2pc_state_log = false; - if (OB_UNLIKELY(is_2pc_blocking())) { - ret = OB_EAGAIN; - TRANS_LOG(WARN, "tx submit log failed because of 2pc blocking", K(ret), KPC(this)); - // It is safe to merge the intermediate_participants because we will block - // the persistent state machine with is_2pc_blocking. The detailed design - // can be found in the implementation of the merge_intermediate_participants. - } else if (is_contain_stat_log(cb_arg_array) && FALSE_IT(is_2pc_state_log = true)) { - } else if (is_2pc_state_log && OB_FAIL(merge_intermediate_participants())) { - TRANS_LOG(WARN, "fail to merge intermediate participants", K(ret), KPC(this)); - } else if (OB_FAIL(ls_tx_ctx_mgr_->get_ls_log_adapter()->submit_log(buf, - size, - base_ts, - cb, - need_nonblock))) { - TRANS_LOG(WARN, "submit log fail", KR(ret), KPC(this)); - } - return ret; -} - int ObPartTransCtx::wait_tx_write_end() { int ret = OB_SUCCESS; @@ -8953,7 +9067,8 @@ int ObPartTransCtx::move_tx_op(const ObTransferMoveTxParam &move_tx_param, // promise tx log before move log if (exec_info_.state_ == ObTxState::INIT) { // promise redo log before move log - if (OB_FAIL(submit_redo_log_())) { + bool submitted = false; + if (OB_FAIL(submit_redo_log_for_freeze_(submitted))) { TRANS_LOG(WARN, "submit log failed", KR(ret), KPC(this)); } else { sub_state_.set_transfer_blocking(); @@ -8994,7 +9109,7 @@ int ObPartTransCtx::move_tx_op(const ObTransferMoveTxParam &move_tx_param, if (epoch_ == arg.epoch_) { trans_expired_time_ = arg.tx_expired_time_; } - update_max_submitted_seq_no(arg.max_submitted_seq_no_); + exec_info_.max_submitted_seq_no_.inc_update(arg.max_submitted_seq_no_); if (arg.last_seq_no_ > last_scn_) { last_scn_.atomic_store(arg.last_seq_no_); } @@ -9020,7 +9135,8 @@ int ObPartTransCtx::move_tx_op(const ObTransferMoveTxParam &move_tx_param, } } if (OB_FAIL(ret)) { - } else if (exec_info_.state_ < ObTxState::COMMIT && OB_FAIL(mt_ctx_.recover_from_table_lock_durable_info(arg.table_lock_info_))) { + } else if (exec_info_.state_ < ObTxState::COMMIT + && OB_FAIL(mt_ctx_.recover_from_table_lock_durable_info(arg.table_lock_info_, true))) { TRANS_LOG(WARN, "recover table lock failed", KR(ret), K(arg)); } else { sub_state_.clear_transfer_blocking(); @@ -9229,5 +9345,154 @@ int ObPartTransCtx::recover_ls_transfer_status_() return ret; } +int ObPartTransCtx::submit_redo_log_out(ObTxLogBlock &log_block, + ObTxLogCb *&log_cb, + ObRedoLogSubmitHelper &helper, + const int64_t replay_hint, + const bool has_hold_ctx_lock, + share::SCN &submitted_scn) +{ + int ret = OB_SUCCESS; + ObTimeGuard time_guard("submit_redo_log_out_"); + CtxLockGuard ctx_lock; + if (!has_hold_ctx_lock) { + get_ctx_guard(ctx_lock, CtxLockGuard::MODE::CTX); + } + bool with_ref = false; + bool alloc_cb = OB_ISNULL(log_cb); + submitted_scn.reset(); + if (alloc_cb && OB_FAIL(prepare_log_cb_(!NEED_FINAL_CB, log_cb))) { + TRANS_LOG(WARN, "get log_cb fail", K(ret), KPC(this)); + } else if (alloc_cb && OB_FAIL(log_cb->reserve_callbacks(helper.callbacks_.count()))) { + TRANS_LOG(WARN, "log cb reserve callbacks space fail", K(ret)); + } else if (OB_FAIL(exec_info_.redo_lsns_.reserve(exec_info_.redo_lsns_.count() + 1))) { + TRANS_LOG(WARN, "reserve memory for redo lsn failed", K(ret)); + } else if (OB_FAIL(acquire_ctx_ref_())) { + TRANS_LOG(ERROR, "acquire ctx ref failed", KR(ret), K(*this)); + } else if (FALSE_IT(with_ref = true)) { + } else if (FALSE_IT(time_guard.click("before_submit_log_block"))) { + } else if (OB_FAIL(submit_log_block_out_(log_block, share::SCN::min_scn(), log_cb, replay_hint))) { + TRANS_LOG(WARN, "submit log to clog adapter failed", KR(ret), K(*this)); + } else { + time_guard.click("submit_out_to_palf"); + submitted_scn = log_cb->get_log_ts(); + ret = after_submit_log_(log_block, log_cb, &helper); + time_guard.click("after_submit"); + log_cb = NULL; // moved + with_ref = false; // moved + } + if (log_cb) { + return_log_cb_(log_cb); + log_cb = NULL; + } + if (with_ref) { + release_ctx_ref_(); + } + TRANS_LOG(DEBUG, "after submit out redo", K(ret), K(time_guard)); + return ret; +} + +bool ObPartTransCtx::is_parallel_logging() const +{ + return exec_info_.serial_final_scn_.is_valid(); +} + +inline bool ObPartTransCtx::has_replay_serial_final_() const +{ + return exec_info_.serial_final_scn_.is_valid() && + exec_info_.max_applied_log_ts_ >= exec_info_.serial_final_scn_; +} + +int ObPartTransCtx::set_replay_completeness(const bool complete) +{ + int ret = OB_SUCCESS; + CtxLockGuard guard(lock_); + if (OB_UNLIKELY(replay_completeness_.is_unknown())) { + if (!complete && !ctx_tx_data_.has_recovered_from_tx_table()) { + if (OB_FAIL(supplement_undo_actions_if_exist_())) { + TRANS_LOG(WARN, "load Undo(s) from tx-table fail", K(ret), KPC(this)); + } else { + TRANS_LOG(INFO, "replay from middle, load Undo(s) from tx-table succuess", + K(ret), K_(ls_id), K_(trans_id)); + } + } + if (OB_SUCC(ret)) { + replay_completeness_.set(complete); + if (!complete) { + force_no_need_replay_checksum_(false, share::SCN::invalid_scn()); + TRANS_LOG(INFO, "incomplete replay, set skip checksum", K_(trans_id), K_(ls_id)); + } + } + } + return ret; +} + +bool ObPartTransCtx::is_replay_completeness_unknown() const +{ + CtxLockGuard guard(lock_); + return replay_completeness_.is_unknown(); +} + +inline bool ObPartTransCtx::is_support_parallel_replay_() const +{ + return cluster_version_accurate_ && cluster_version_ >= CLUSTER_VERSION_4_3_0_0; +} + +inline void ObPartTransCtx::switch_to_parallel_logging_(const share::SCN serial_final_scn) +{ + // when start replaying serial final redo log or submitted serial final redo log + // switch the Tx's logging mode to parallel logging + // this include mark serial final scn point in exec_info_ + // and notify callback_mgr to remember the serial_final_scn + // which used to for check whether the callback-list has replayed continuously + // or all of it's serial logs has been synced continously + // if reach these condition, the checksum calculations of callback-list can continues + // into the parallel logged part + exec_info_.serial_final_scn_.atomic_set(serial_final_scn); + // remember the max of seq_no of redos currently submitted + // if an rollback to savepoint before this point, which means + // replay of this rollback-savepoint-log must pre-berrier to + // wait serial replay parts finished + exec_info_.serial_final_seq_no_ = exec_info_.max_submitted_seq_no_; + mt_ctx_.set_parallel_logging(serial_final_scn); +} + +inline void ObPartTransCtx::recovery_parallel_logging_() +{ + mt_ctx_.set_parallel_logging(exec_info_.serial_final_scn_); + if (exec_info_.max_applied_log_ts_ >= exec_info_.serial_final_scn_) { + // the serial final log has been synced or replayed + // notify callback_mgr serial part is finished + // by fake an replay success call + mt_ctx_.replay_end(true, 0, exec_info_.serial_final_scn_); + } +} + +inline int ObPartTransCtx::correct_cluster_version_(uint64_t cluster_version_in_log) +{ + int ret = OB_SUCCESS; + if (cluster_version_in_log > 0 && cluster_version_ != cluster_version_in_log) { + if (cluster_version_accurate_) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "cluster_version is different with TxCtx", K(ret), KPC(this)); + } else { + cluster_version_ = cluster_version_in_log; + cluster_version_accurate_ = true; + } + } + return ret; +} + +int ObPartTransCtx::get_stat_for_virtual_table(share::ObLSArray &participants, int &busy_cbs_cnt) +{ + int ret = OB_SUCCESS; + if (OB_SUCC(lock_.try_rdlock_ctx())) { + participants.assign(exec_info_.participants_); + busy_cbs_cnt = busy_cbs_.get_size(); + lock_.unlock_ctx(); + } + return ret; +} + } // namespace transaction } // namespace oceanbase diff --git a/src/storage/tx/ob_trans_part_ctx.h b/src/storage/tx/ob_trans_part_ctx.h index 4a1f2195a7..22507e8ea7 100644 --- a/src/storage/tx/ob_trans_part_ctx.h +++ b/src/storage/tx/ob_trans_part_ctx.h @@ -237,14 +237,8 @@ public: uint64_t get_lock_for_read_retry_count() const { return mt_ctx_.get_lock_for_read_retry_count(); } int check_scheduler_status(); - int remove_callback_for_uncommited_txn( - const memtable::ObMemtableSet *memtable_set); + int remove_callback_for_uncommited_txn(const memtable::ObMemtableSet *memtable_set); int64_t get_trans_mem_total_size() const { return mt_ctx_.get_trans_mem_total_size(); } - - void update_max_submitted_seq_no(const ObTxSEQ seq_no) - { - exec_info_.max_submitted_seq_no_.inc_update(seq_no); - } int check_with_tx_data(ObITxDataCheckFunctor &fn); const share::SCN get_rec_log_ts() const; int on_tx_ctx_table_flushed(); @@ -260,8 +254,6 @@ public: void set_table_lock_killed() { mt_ctx_.set_table_lock_killed(); } bool is_table_lock_killed() const; - bool need_update_schema_version(const int64_t log_id, - const share::SCN log_ts); share::ObLSID get_ls_id() const { return ls_id_; } @@ -297,8 +289,8 @@ private: K(role_state_), K(start_replay_ts_), K(start_recover_ts_), - K(is_incomplete_replay_ctx_), K(epoch_), + K(replay_completeness_), K(mt_ctx_), K(coord_prepare_info_arr_), K_(upstream_state), @@ -312,7 +304,8 @@ private: K_(lastest_snapshot), K_(state_info_array), K_(last_request_ts), - KP_(block_frozen_memtable)); + KP_(block_frozen_memtable), + K_(max_2pc_commit_scn)); public: static const int64_t OP_LOCAL_NUM = 16; static const int64_t RESERVED_MEM_SIZE = 256; @@ -341,7 +334,7 @@ private: int trans_replay_commit_(const share::SCN &commit_version, const share::SCN &final_log_ts, const uint64_t log_cluster_version, - const int64_t checksum); + const uint64_t checksum); int trans_replay_abort_(const share::SCN &final_log_ts); int update_publish_version_(const share::SCN &publish_version, const bool for_replay); bool can_be_recycled_(); @@ -353,9 +346,6 @@ private: int on_success_ops_(ObTxLogCb * log_cb); void check_and_register_timeout_task_(); int recover_ls_transfer_status_(); - - // bool need_commit_barrier(); - public: // ======================================================== // newly added for 4.0 @@ -370,26 +360,33 @@ public: virtual int submit_log(const ObTwoPhaseCommitLogType &log_type) override; int try_submit_next_log(); // for instant logging and freezing - int submit_redo_log(const bool is_freeze); - - int push_repalying_log_ts(const share::SCN log_ts_ns); - int push_replayed_log_ts(const share::SCN log_ts_ns, const palf::LSN &offset); - + int submit_redo_after_write(const bool force, const ObTxSEQ &write_seq_no); + int submit_redo_log_for_freeze(); + int return_redo_log_cb(ObTxLogCb *log_cb); + int push_repalying_log_ts(const share::SCN log_ts_ns, const bool is_first); + int push_replayed_log_ts(const share::SCN log_ts_ns, + const palf::LSN &offset, + const int64_t log_entry_no); int iter_next_log_for_replay(ObTxLogBlock &log_block, ObTxLogHeader &log_header, const share::SCN log_scn); int replay_one_part_of_big_segment(const palf::LSN &offset, const share::SCN ×tamp, const int64_t &part_log_no); - + int set_replay_completeness(const bool complete); + bool is_replay_completeness_unknown() const; int replay_redo_in_ctx(const ObTxRedoLog &redo_log, const palf::LSN &offset, const share::SCN ×tamp, - const int64_t &part_log_no); + const int64_t &part_log_no, + const bool is_tx_log_queue, + const bool serial_final); int replay_rollback_to(const ObTxRollbackToLog &log, const palf::LSN &offset, const share::SCN ×tamp, - const int64_t &part_log_no); + const int64_t &part_log_no, + const bool is_tx_log_queue, + const bool pre_barrier); int replay_active_info(const ObTxActiveInfoLog &active_info_log, const palf::LSN &offset, const share::SCN ×tamp, @@ -397,7 +394,8 @@ public: int replay_commit_info(const ObTxCommitInfoLog &commit_info_log, const palf::LSN &offset, const share::SCN ×tamp, - const int64_t &part_log_no); + const int64_t &part_log_no, + const bool pre_barrier); int replay_prepare(const ObTxPrepareLog &prepare_log, const palf::LSN &offset, const share::SCN ×tamp, @@ -426,11 +424,9 @@ public: const share::SCN ×tamp, const int64_t &part_log_no); - void force_no_need_replay_checksum(); + void force_no_need_replay_checksum(const bool parallel_replay, const share::SCN &log_ts); - void check_no_need_replay_checksum(const share::SCN &log_ts); - - int validate_replay_log_entry_no(bool first_created_ctx, int64_t log_entry_no, const share::SCN &log_ts); + void check_no_need_replay_checksum(const share::SCN &log_ts, const int index); // return the min log ts of those logs which are submitted but // not callbacked yet, if there is no such log return INT64_MAX @@ -443,6 +439,8 @@ public: int serialize_tx_ctx_to_buffer(ObTxLocalBuffer &buffer, int64_t &serialize_size); int recover_tx_ctx_table_info(ObTxCtxTableInfo &ctx_info); + int correct_cluster_version_(uint64_t cluster_version_in_log); + // leader switch related bool need_callback_scheduler_(); int switch_to_follower_forcedly(ObIArray &cb_array); @@ -486,11 +484,16 @@ private: // ======================================================== // newly added for 4.0 int submit_log_impl_(const ObTxLogType log_type); + void handle_submit_log_err_(const ObTxLogType log_type, int &ret); + typedef logservice::ObReplayBarrierType ObReplayBarrierType; + int submit_log_block_out_(ObTxLogBlock &block, + const share::SCN &base_scn, + ObTxLogCb *&log_cb, + const int64_t replay_hint = 0, + const ObReplayBarrierType barrier = ObReplayBarrierType::NO_NEED_BARRIER); int after_submit_log_(ObTxLogBlock &log_block, ObTxLogCb *log_cb, memtable::ObRedoLogSubmitHelper *redo_helper); - - int submit_redo_log_(); int submit_commit_log_(); int submit_abort_log_(); int submit_prepare_log_(); @@ -498,15 +501,24 @@ private: int submit_record_log_(); int submit_redo_commit_info_log_(); int submit_redo_active_info_log_(); - int submit_redo_log_(ObTxLogBlock &log_block, - bool &has_redo, - memtable::ObRedoLogSubmitHelper &helper); + int submit_redo_if_serial_logging_(ObTxLogBlock &log_block, + bool &has_redo, + memtable::ObRedoLogSubmitHelper &helper); + int submit_redo_if_parallel_logging_(); int submit_redo_commit_info_log_(ObTxLogBlock &log_block, bool &has_redo, - memtable::ObRedoLogSubmitHelper &helper); - - int submit_pending_log_block_(ObTxLogBlock &log_block, memtable::ObRedoLogSubmitHelper &helper); - + memtable::ObRedoLogSubmitHelper &helper, + logservice::ObReplayBarrierType &barrier); + int submit_pending_log_block_(ObTxLogBlock &log_block, + memtable::ObRedoLogSubmitHelper &helper, + const logservice::ObReplayBarrierType &barrier); + bool should_switch_to_parallel_logging_(); + void switch_to_parallel_logging_(const share::SCN serial_final_scn); + bool has_replay_serial_final_() const; + void recovery_parallel_logging_(); + int check_can_submit_redo_(); + void force_no_need_replay_checksum_(const bool parallel_replay, const share::SCN &log_ts); + int serial_submit_redo_after_write_(); int submit_big_segment_log_(); int prepare_big_segment_submit_(ObTxLogCb *segment_cb, const share::SCN &base_scn, @@ -515,7 +527,10 @@ private: int add_unsynced_segment_cb_(ObTxLogCb *log_cb); int remove_unsynced_segment_cb_(const share::SCN &remove_scn); share::SCN get_min_unsyncd_segment_scn_(); - + int init_log_block_(ObTxLogBlock &log_block, + const int64_t suggested_buf_size = ObTxAdaptiveLogBuf::NORMAL_LOG_BUF_SIZE, + const bool serial_final = false); + int reuse_log_block_(ObTxLogBlock &log_block); int compensate_abort_log_(); int validate_commit_info_log_(const ObTxCommitInfoLog &commit_info_log); @@ -523,11 +538,6 @@ private: ObTxLogType &ret_log_type); int switch_log_type_(const ObTxLogType ret_log_type, ObTwoPhaseCommitLogType &log_type); - - int fill_redo_log_(char *buf, - const int64_t buf_len, - int64_t &pos, - memtable::ObRedoLogSubmitHelper &helper); int64_t get_redo_log_no_() const; bool has_persisted_log_() const; @@ -624,6 +634,11 @@ private: int errism_dup_table_redo_sync_(); int errism_submit_prepare_log_(); + int replay_redo_in_ctx_compat_(const ObTxRedoLog &redo_log, + const palf::LSN &offset, + const share::SCN ×tamp, + const int64_t &part_log_no); + bool is_support_parallel_replay_() const; int errsim_notify_mds_(); protected: virtual int get_gts_(share::SCN >s); @@ -634,12 +649,12 @@ protected: private: int init_log_cbs_(const share::ObLSID&ls_id, const ObTransID &tx_id); - int extend_log_cbs_(); + int extend_log_cbs_(ObTxLogCb *&log_cb); void reset_log_cb_list_(common::ObDList &cb_list); void reset_log_cbs_(); int prepare_log_cb_(const bool need_final_cb, ObTxLogCb *&log_cb); int get_log_cb_(const bool need_final_cb, ObTxLogCb *&log_cb); - int return_log_cb_(ObTxLogCb *log_cb); + int return_log_cb_(ObTxLogCb *log_cb, bool release_final_cb = false); int get_max_submitting_log_info_(palf::LSN &lsn, share::SCN &log_ts); int get_prev_log_lsn_(const ObTxLogBlock &log_block, ObTxLogType prev_log_type, palf::LSN &lsn); int set_start_scn_in_commit_log_(ObTxCommitLog &commit_log); @@ -722,7 +737,7 @@ private: ObITransRpc* rpc, const bool ls_deleted); static int get_max_decided_scn_(const share::ObLSID &ls_id, share::SCN &scn); - int get_2pc_participants_copy(share::ObLSArray ©_participants); + int get_stat_for_virtual_table(share::ObLSArray &participants, int &busy_cbs_cnt); // for xa int post_tx_sub_prepare_resp_(const int status); int post_tx_sub_commit_resp_(const int status); @@ -823,10 +838,11 @@ public: int check_status(); /* * start_access - start txn protected resources access - * @data_seq: the sequence_no of current access + * @data_seq: the sequence_no of current access will be alloced * new created data will marked with this seq no + * @branch: branch id of this access */ - int start_access(const ObTxDesc &tx_desc, ObTxSEQ &data_seq); + int start_access(const ObTxDesc &tx_desc, ObTxSEQ &data_seq, const int16_t branch); /* * end_access - end of txn protected resources access */ @@ -843,8 +859,12 @@ private: int tx_keepalive_response_(const int64_t status); void post_keepalive_msg_(const int status); void notify_scheduler_tx_killed_(const int kill_reason); - int rollback_to_savepoint_(const ObTxSEQ from_scn, const ObTxSEQ to_scn); - int submit_rollback_to_log_(const ObTxSEQ from_scn, const ObTxSEQ to_scn, ObTxData *tx_data); + int rollback_to_savepoint_(const ObTxSEQ from_scn, + const ObTxSEQ to_scn, + const share::SCN replay_scn = share::SCN::invalid_scn()); + int submit_rollback_to_log_(const ObTxSEQ from_scn, + const ObTxSEQ to_scn, + ObTxData *tx_data); int set_state_info_array_(); int update_state_info_array_(const ObStateInfo& state_info); int update_state_info_array_with_transfer_parts_(const ObTxCommitParts &parts, const ObLSID &ls_id); @@ -853,9 +873,19 @@ private: const share::ObLSID &ori_ls_id, const ObAddr &ori_addr); int check_ls_state_(const SCN &snapshot, const ObLSID &ls_id, const ObStandbyCheckInfo &check_info); int get_ls_replica_readable_scn_(const ObLSID &ls_id, SCN &snapshot_version); - int check_and_submit_redo_log_(bool &try_submit); int submit_redo_log_for_freeze_(bool &try_submit); void print_first_mvcc_callback_(); +public: + int prepare_for_submit_redo(ObTxLogCb *&log_cb, + ObTxLogBlock &log_block, + const bool serial_final = false); + int submit_redo_log_out(ObTxLogBlock &log_block, + ObTxLogCb *&log_cb, + memtable::ObRedoLogSubmitHelper &helper, + const int64_t replay_hint, + const bool has_hold_ctx_lock, + share::SCN &submitted_scn); + bool is_parallel_logging() const; int assign_commit_parts(const share::ObLSArray &log_participants, const ObTxCommitParts &log_commit_parts); protected: @@ -907,7 +937,12 @@ private: int64_t last_ask_scheduler_status_ts_; int64_t cur_query_start_time_; - + // when cluster_version is unknown at ctx created time, will choice + // CLUSTER_CURRENT_VERSION, which may not the real cluster_version + // of this transaction + // this can only happen when create ctx for replay and create ctx + // for recovery before v.4.3 + bool cluster_version_accurate_; /* * used during txn protected data access */ @@ -941,6 +976,7 @@ private: common::ObDList free_cbs_; common::ObDList busy_cbs_; ObTxLogCb final_log_cb_; + ObSpinLock log_cb_lock_; ObTxLogBigSegmentInfo big_segment_info_; // flag if the first callback is linked to a logging_block memtable // to prevent unnecessary submit_log actions for freeze @@ -970,7 +1006,16 @@ private: // | start_log_ts = n | recover_ts = n | remove from tx_ctx_table & dump | recover_ts = n+10 | crash | | min_ckpt_ts n+m | | tx_ctx is incomplete | // | end_log_ts = n+10 | ----------------> | | -------------------> | | --> | (0 | | // +-------------------+ +---------------------------------+ +-------+ +-----------------+ +----------------------+ - bool is_incomplete_replay_ctx_; + struct ReplayCompleteness { + ReplayCompleteness(): complete_(C::UNKNOWN) {} + void reset() { complete_ = C::UNKNOWN; } + enum class C : int { COMPLETE = 1, INCOMPLETE = 0, UNKNOWN = -1 } complete_; + void set(const bool complete) { complete_ = complete ? C::COMPLETE : C::INCOMPLETE; } + bool is_unknown() const { return complete_ == C::UNKNOWN; } + bool is_complete() const { return complete_ == C::COMPLETE; } + bool is_incomplete() const { return complete_ == C::INCOMPLETE; } + DECLARE_TO_STRING { int64_t pos = 0; BUF_PRINTF("%d", complete_); return pos; }; + } replay_completeness_; // set true when submitting redo log for freezing and reset after freezing bool is_submitting_redo_log_for_freeze_; share::SCN start_replay_ts_; // replay debug diff --git a/src/storage/tx/ob_trans_rpc.h b/src/storage/tx/ob_trans_rpc.h index 6d8a8199be..6306d42b4f 100644 --- a/src/storage/tx/ob_trans_rpc.h +++ b/src/storage/tx/ob_trans_rpc.h @@ -282,7 +282,7 @@ int ObTxRPCCB::process() } } } - if (OB_SUCCESS != ret || (OB_SUCCESS != status && status != -1)) { + if (OB_SUCCESS != ret || (OB_SUCCESS != status && status != -1 && status != OB_NEED_RETRY)) { TRANS_LOG(WARN, "trx rpc callback", K(ret), K(status), K(dst), K(result)); } return ret; @@ -338,7 +338,8 @@ void ObTxRPCCB::on_timeout() if (transaction::ObTxMsgTypeChecker::is_2pc_msg_type(msg_type_)) { // do nothing } else { - if (receiver_ls_id_.is_valid()) { + if (receiver_ls_id_.is_scheduler_ls()) { + } else if (receiver_ls_id_.is_valid()) { if (OB_FAIL(refresh_location_cache(receiver_ls_id_))) { TRANS_LOG(WARN, "refresh location cache error", KR(ret), K_(trans_id), K_(receiver_ls_id), K(dst), K_(tenant_id)); } else { diff --git a/src/storage/tx/ob_trans_service_v4.cpp b/src/storage/tx/ob_trans_service_v4.cpp index f8d35f4cd4..b2fdf6d259 100644 --- a/src/storage/tx/ob_trans_service_v4.cpp +++ b/src/storage/tx/ob_trans_service_v4.cpp @@ -1099,6 +1099,7 @@ int ObTransService::get_write_store_ctx(ObTxDesc &tx, int ret = OB_SUCCESS; const share::ObLSID &ls_id = store_ctx.ls_id_; ObPartTransCtx *tx_ctx = NULL; + const int16_t branch = store_ctx.branch_; ObTxSEQ data_scn = spec_seq_no; // for LOB aux table, spec_seq_no is valid ObTxSnapshot snap = snapshot.core_; ObTxTableGuard tx_table_guard; @@ -1122,7 +1123,7 @@ int ObTransService::get_write_store_ctx(ObTxDesc &tx, TRANS_LOG(WARN, "use ls snapshot access another ls", K(ret), K(snapshot), K(ls_id)); } else if (OB_FAIL(acquire_tx_ctx(ls_id, tx, tx_ctx, store_ctx.ls_, special))) { TRANS_LOG(WARN, "acquire tx ctx fail", K(ret), K(tx), K(ls_id), KPC(this)); - } else if (OB_FAIL(tx_ctx->start_access(tx, data_scn))) { + } else if (OB_FAIL(tx_ctx->start_access(tx, data_scn, branch))) { TRANS_LOG(WARN, "tx ctx start access fail", K(ret), K(tx_ctx), K(ls_id), KPC(this)); // when transfer move_tx phase we put src_ls tx_ctx into dest_ls ctx_mgr when transfer abort we need remove it // when access tx_ctx first get ctx from mgr, second increase pending_write @@ -1133,7 +1134,7 @@ int ObTransService::get_write_store_ctx(ObTxDesc &tx, ob_usleep(10 * 1000); if (OB_FAIL(acquire_tx_ctx(ls_id, tx, tx_ctx, store_ctx.ls_, special))) { TRANS_LOG(WARN, "acquire tx ctx fail", K(ret), K(tx), K(ls_id), KPC(this)); - } else if (OB_FAIL(tx_ctx->start_access(tx, data_scn))) { + } else if (OB_FAIL(tx_ctx->start_access(tx, data_scn, branch))) { TRANS_LOG(WARN, "tx ctx start access fail", K(ret), K(tx_ctx), K(ls_id), KPC(this)); } } @@ -2057,6 +2058,7 @@ int ObTransService::handle_sp_rollback_request(ObTxRollbackSPMsg &msg, msg.epoch_, msg.op_sn_, msg.savepoint_, + msg.tx_seq_base_, ctx_born_epoch, msg.tx_ptr_, msg.for_transfer(), @@ -2747,6 +2749,7 @@ int ObTransService::recover_tx(const ObTxInfo &tx_info, ObTxDesc *&tx) tx->flags_.EXPLICIT_ = true; tx->tenant_id_ = tx_info.tenant_id_; tx->cluster_id_ = tx_info.cluster_id_; + tx->seq_base_ = tx_info.seq_base_; tx->cluster_version_ = tx_info.cluster_version_; tx->addr_ = tx_info.addr_; /*origin scheduler addr*/ tx->tx_id_ = tx_info.tx_id_; @@ -2779,6 +2782,7 @@ int ObTransService::get_tx_info(ObTxDesc &tx, ObTxInfo &tx_info) tx_info.tenant_id_ = tx.tenant_id_; tx_info.cluster_id_ = tx.cluster_id_; tx_info.cluster_version_ = tx.cluster_version_; + tx_info.seq_base_ = tx.seq_base_; tx_info.addr_ = tx.addr_; tx_info.tx_id_ = tx.tx_id_; tx_info.isolation_ = tx.isolation_; diff --git a/src/storage/tx/ob_trans_service_v4.h b/src/storage/tx/ob_trans_service_v4.h index 10a0eebb6c..5a0bad7c5b 100644 --- a/src/storage/tx/ob_trans_service_v4.h +++ b/src/storage/tx/ob_trans_service_v4.h @@ -195,7 +195,7 @@ TO_STRING_KV(K(is_inited_), K(tenant_id_), KP(this)); private: int check_ls_status_(const share::ObLSID &ls_id, bool &leader); -void init_tx_(ObTxDesc &tx, const uint32_t session_id); +int init_tx_(ObTxDesc &tx, const uint32_t session_id); int start_tx_(ObTxDesc &tx); int abort_tx_(ObTxDesc &tx, const int cause, bool cleanup = true); void abort_tx__(ObTxDesc &tx, const bool cleanup); @@ -353,6 +353,7 @@ int ls_rollback_to_savepoint_(const ObTransID &tx_id, const int64_t verify_epoch, const int64_t op_sn, const ObTxSEQ savepoint, + const int64_t tx_seq_base, int64_t &ctx_born_epoch, const ObTxDesc *tx, const bool for_transfer, @@ -382,6 +383,7 @@ int rollback_to_global_implicit_savepoint_(ObTxDesc &tx, int ls_sync_rollback_savepoint__(ObPartTransCtx *part_ctx, const ObTxSEQ savepoint, const int64_t op_sn, + const int64_t tx_seq_base, const int64_t expire_ts, const ObTxSEQ specified_from_scn, ObIArray &downstream_parts); diff --git a/src/storage/tx/ob_trans_submit_log_cb.cpp b/src/storage/tx/ob_trans_submit_log_cb.cpp index 3e5e2e7910..79abef12f0 100644 --- a/src/storage/tx/ob_trans_submit_log_cb.cpp +++ b/src/storage/tx/ob_trans_submit_log_cb.cpp @@ -148,20 +148,9 @@ int ObTxLogCb::on_success() ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "ctx is null", K(ret), K(tx_id), KP(ctx_)); } else { - // make sure set log type to log callback successfully - ctx_->test_lock(this); - if (cb_arg_array_.count() == 0) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "cb arg array is empty", K(ret), K(tx_id), KP(ctx_)); - ctx_->print_trace_log(); - } else { - TRANS_LOG(DEBUG, "get last log type success", K(tx_id)); - // TODO. iterate all log type - ObPartTransCtx *part_ctx = static_cast(ctx_); - - if (OB_FAIL(part_ctx->on_success(this))) { - TRANS_LOG(WARN, "sync log success callback error", K(ret), K(tx_id)); - } + ObPartTransCtx *part_ctx = static_cast(ctx_); + if (OB_FAIL(part_ctx->on_success(this))) { + TRANS_LOG(WARN, "sync log success callback error", K(ret), K(tx_id)); } } @@ -179,19 +168,9 @@ int ObTxLogCb::on_failure() ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "ctx is null", KR(ret), K(*this)); } else { - // make sure set log type to log callback successfully - ctx_->test_lock(this); - if (cb_arg_array_.count() == 0) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "cb arg array is empty", K(ret)); - } else { - TRANS_LOG(DEBUG, "get last log type success", K(tx_id)); - // TODO. iterate all log type - ObPartTransCtx *part_ctx = static_cast(ctx_); - - if (OB_FAIL(part_ctx->on_failure(this))) { - TRANS_LOG(WARN, "sync log success callback error", KR(ret), K(tx_id)); - } + ObPartTransCtx *part_ctx = static_cast(ctx_); + if (OB_FAIL(part_ctx->on_failure(this))) { + TRANS_LOG(WARN, "sync log success callback error", KR(ret), K(tx_id)); } } TRANS_LOG(INFO, "ObTxLogCb::on_failure end", KR(ret), K(tx_id)); @@ -208,11 +187,11 @@ int ObTxLogCb::copy(const ObTxLogCb &other) ls_id_ = other.ls_id_; trans_id_ = other.trans_id_; ctx_ = other.ctx_; - callbacks_ = other.callbacks_; - is_callbacked_ = other.is_callbacked_; - + if (OB_FAIL(callbacks_.assign(other.callbacks_))) { + TRANS_LOG(WARN, "assign callbacks failed", K(ret), KPC(this)); + } else if (FALSE_IT(is_callbacked_ = other.is_callbacked_)) { // without txdata - if (OB_FAIL(mds_range_.assign(other.mds_range_))) { + } else if (OB_FAIL(mds_range_.assign(other.mds_range_))) { TRANS_LOG(WARN, "assign mds range failed", K(ret), KPC(this)); } else if (OB_FAIL(cb_arg_array_.assign(other.cb_arg_array_))) { TRANS_LOG(WARN, "assign cb_arg_array_ failed", K(ret), KPC(this)); diff --git a/src/storage/tx/ob_trans_submit_log_cb.h b/src/storage/tx/ob_trans_submit_log_cb.h index acfc9813dd..03d4fc9f59 100644 --- a/src/storage/tx/ob_trans_submit_log_cb.h +++ b/src/storage/tx/ob_trans_submit_log_cb.h @@ -69,7 +69,7 @@ protected: palf::LSN lsn_; int64_t submit_ts_; }; - +typedef common::ObSEArray ObCallbackScopeArray; class ObTxLogCb : public ObTxBaseLogCb, public common::ObDLinkBase { @@ -94,10 +94,11 @@ public: } } ObTxData* get_tx_data() { return tx_data_guard_.tx_data(); } - void set_callbacks(const memtable::ObCallbackScope &callbacks) { callbacks_ = callbacks; } - memtable::ObCallbackScope& get_callbacks() { return callbacks_; } - void set_callbacked() { is_callbacked_ = true; } - bool is_callbacked() const { return is_callbacked_; } + int set_callbacks(const ObCallbackScopeArray &callbacks) { return callbacks_.assign(callbacks); } + ObCallbackScopeArray& get_callbacks() { return callbacks_; } + int reserve_callbacks(int cnt) { return callbacks_.reserve(cnt); } + void set_callbacked() { ATOMIC_STORE(&is_callbacked_, true); } + bool is_callbacked() const { return ATOMIC_LOAD(&is_callbacked_); } bool is_dynamic() const { return is_dynamic_; } ObTxCbArgArray &get_cb_arg_array() { return cb_arg_array_; } const ObTxCbArgArray &get_cb_arg_array() const { return cb_arg_array_; } @@ -126,7 +127,8 @@ public: K(is_dynamic_), K(mds_range_), K(cb_arg_array_), - K(first_part_scn_)); + K(first_part_scn_), + K(callbacks_.count())); private: DISALLOW_COPY_AND_ASSIGN(ObTxLogCb); private: @@ -135,7 +137,7 @@ private: ObTransID trans_id_; ObTransCtx *ctx_; ObTxDataGuard tx_data_guard_; - memtable::ObCallbackScope callbacks_; + ObCallbackScopeArray callbacks_; bool is_callbacked_; bool is_dynamic_; ObTxMDSRange mds_range_; diff --git a/src/storage/tx/ob_tx_2pc_msg_handler.cpp b/src/storage/tx/ob_tx_2pc_msg_handler.cpp index 6108bb516f..39ca990729 100644 --- a/src/storage/tx/ob_tx_2pc_msg_handler.cpp +++ b/src/storage/tx/ob_tx_2pc_msg_handler.cpp @@ -430,23 +430,6 @@ int ObPartTransCtx::set_2pc_participants_(const ObTxCommitParts& participants) return ret; } -int ObPartTransCtx::get_2pc_participants_copy(ObLSArray ©_participants) -{ - int ret = OB_SUCCESS; - - if (OB_FAIL(lock_.try_lock())) { - TRANS_LOG(INFO, "get participants copy fail", K_(trans_id)); - // rewrite - ret = OB_SUCCESS; - } else { - if (OB_SUCCESS != (ret = copy_participants.assign(exec_info_.participants_))) { - TRANS_LOG(WARN, "ObPartTransCtx get participants copy error", K(ret), K(*this)); - } - lock_.unlock(); - } - return ret; -} - int ObPartTransCtx::set_2pc_request_id_(const int64_t request_id) { int ret = OB_SUCCESS; diff --git a/src/storage/tx/ob_tx_api.cpp b/src/storage/tx/ob_tx_api.cpp index 43bdba5459..2bace002eb 100644 --- a/src/storage/tx/ob_tx_api.cpp +++ b/src/storage/tx/ob_tx_api.cpp @@ -60,8 +60,9 @@ using namespace share; namespace transaction { -inline void ObTransService::init_tx_(ObTxDesc &tx, const uint32_t session_id) +inline int ObTransService::init_tx_(ObTxDesc &tx, const uint32_t session_id) { + int ret = OB_SUCCESS; tx.tenant_id_ = tenant_id_; tx.addr_ = self_; tx.sess_id_ = session_id; @@ -70,6 +71,12 @@ inline void ObTransService::init_tx_(ObTxDesc &tx, const uint32_t session_id) tx.expire_ts_ = INT64_MAX; tx.op_sn_ = 1; tx.state_ = ObTxDesc::State::IDLE; + if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id_, tx.cluster_version_))) { + TRANS_LOG(WARN, "get min data version fail", K(ret), K(tx)); + } else if (tx.cluster_version_ >= DATA_VERSION_4_3_0_0) { + tx.seq_base_ = common::ObSequence::get_max_seq_no() - 1; + } + return ret; } int ObTransService::acquire_tx(ObTxDesc *&tx, const uint32_t session_id) @@ -78,7 +85,7 @@ int ObTransService::acquire_tx(ObTxDesc *&tx, const uint32_t session_id) if (OB_FAIL(tx_desc_mgr_.alloc(tx))) { TRANS_LOG(WARN, "alloc tx fail", K(ret)); } else { - init_tx_(*tx, session_id); + ret = init_tx_(*tx, session_id); } TRANS_LOG(TRACE, "acquire tx", KPC(tx), K(session_id)); if (OB_SUCC(ret)) { @@ -196,7 +203,7 @@ int ObTransService::reuse_tx(ObTxDesc &tx) // it is safe to operate tx without lock when not shared uint32_t session_id = tx.sess_id_; tx.reset(); - init_tx_(tx, session_id); + ret = init_tx_(tx, session_id); } TRANS_LOG(TRACE, "reuse tx", K(ret), K(orig_tx_id), K(tx)); ObTransTraceLog &tlog = tx.get_tlog(); @@ -260,7 +267,6 @@ int ObTransService::start_tx(ObTxDesc &tx, const ObTxParam &tx_param, const ObTr if (OB_FAIL(ret)) { TRANS_LOG(WARN, "add tx to txMgr fail", K(ret), K(tx)); } else { - tx.cluster_version_ = GET_MIN_CLUSTER_VERSION(); tx.cluster_id_ = tx_param.cluster_id_; tx.access_mode_ = tx_param.access_mode_; tx.isolation_ = tx_param.isolation_; @@ -892,6 +898,27 @@ void ObTransService::unregister_tx_snapshot_verify(ObTxReadSnapshot &snapshot) TRANS_LOG(TRACE, "unreigster snapshot", K(ret), K(snapshot)); } +int ObTransService::create_branch_savepoint(ObTxDesc &tx, + const int16_t branch, + ObTxSEQ &savepoint) +{ + int ret = OB_SUCCESS; + ObSpinLockGuard guard(tx.lock_); + if (OB_SUCC(tx_sanity_check_(tx))) { + savepoint = tx.inc_and_get_tx_seq(branch); + TRANS_LOG(TRACE, "create branch savepoint", K(ret), K(branch), K(savepoint), K(tx)); + ObTransTraceLog &tlog = tx.get_tlog(); + REC_TRANS_TRACE_EXT(&tlog, create_branch_savepoint, + OB_Y(ret), + OB_ID(savepoint), savepoint.cast_to_int(), + OB_ID(branch), (int)branch, + OB_ID(opid), tx.op_sn_, + OB_ID(ref), tx.get_ref(), + OB_ID(thread_id), GETTID()); + } + return ret; +} + int ObTransService::create_implicit_savepoint(ObTxDesc &tx, const ObTxParam &tx_param, ObTxSEQ &savepoint, @@ -924,7 +951,9 @@ int ObTransService::create_local_implicit_savepoint_(ObTxDesc &tx, TRANS_LOG(TRACE, "create local implicit savepoint", K(ret), K(savepoint), K(tx)); ObTransTraceLog &tlog = tx.get_tlog(); REC_TRANS_TRACE_EXT(&tlog, create_local_implicit_savepoint, - OB_Y(ret), OB_ID(savepoint), savepoint.cast_to_int(), OB_ID(opid), tx.op_sn_, + OB_Y(ret), + OB_ID(savepoint), savepoint.cast_to_int(), + OB_ID(opid), tx.op_sn_, OB_ID(ref), tx.get_ref(), OB_ID(thread_id), GETTID()); return ret; @@ -937,7 +966,6 @@ int ObTransService::create_global_implicit_savepoint_(ObTxDesc &tx, { int ret = OB_SUCCESS; if (tx.state_ == ObTxDesc::State::IDLE) { - tx.cluster_version_ = GET_MIN_CLUSTER_VERSION(); tx.cluster_id_ = tx_param.cluster_id_; tx.access_mode_ = tx_param.access_mode_; tx.timeout_us_ = tx_param.timeout_us_; @@ -946,29 +974,33 @@ int ObTransService::create_global_implicit_savepoint_(ObTxDesc &tx, tx.snapshot_version_.reset(); // invalidate previouse snapshot } } - // NOTE: the lock_timeout_us_ can be changed even tx active - tx.lock_timeout_us_ = tx_param.lock_timeout_us_; - - tx.inc_op_sn(); - savepoint = tx.inc_and_get_tx_seq(0); - if (tx.state_ == ObTxDesc::State::IDLE && !tx.tx_id_.is_valid()) { - if (tx.has_implicit_savepoint()) { - ret = OB_TRANS_INVALID_STATE; - TRANS_LOG(WARN, "has implicit savepoint, but tx_id is invalid", K(ret), K(tx)); - } else if (OB_FAIL(tx_desc_mgr_.add(tx))) { - TRANS_LOG(WARN, "failed to register with txMgr", K(ret), K(tx)); + if (OB_SUCC(ret)) { + // NOTE: the lock_timeout_us_ can be changed even tx active + tx.lock_timeout_us_ = tx_param.lock_timeout_us_; + tx.inc_op_sn(); + savepoint = tx.inc_and_get_tx_seq(0); + if (tx.state_ == ObTxDesc::State::IDLE && !tx.tx_id_.is_valid()) { + if (tx.has_implicit_savepoint()) { + ret = OB_TRANS_INVALID_STATE; + TRANS_LOG(WARN, "has implicit savepoint, but tx_id is invalid", K(ret), K(tx)); + } else if (OB_FAIL(tx_desc_mgr_.add(tx))) { + TRANS_LOG(WARN, "failed to register with txMgr", K(ret), K(tx)); + } } } if (OB_SUCC(ret)) { if (release) { tx.release_all_implicit_savepoint(); + // reset branch_id alloc for further writes + tx.last_branch_id_ = tx.branch_id_offset() - 1; } tx.add_implicit_savepoint(savepoint); } ObTransTraceLog &tlog = tx.get_tlog(); REC_TRANS_TRACE_EXT(&tlog, create_global_implicit_savepoint, OB_Y(ret), OB_ID(txid), tx.tx_id_, - OB_ID(savepoint), savepoint.cast_to_int(), OB_Y(release), + OB_ID(savepoint), savepoint.cast_to_int(), + OB_Y(release), OB_ID(opid), tx.op_sn_, OB_ID(ref), tx.get_ref(), OB_ID(thread_id), GETTID()); @@ -1003,7 +1035,8 @@ int ObTransService::rollback_to_implicit_savepoint(ObTxDesc &tx, int ret = OB_SUCCESS; ObSpinLockGuard guard(tx.lock_); if (OB_FAIL(tx_sanity_check_(tx))) { - } else if (tx.flags_.SHADOW_) { + } else if (savepoint.get_branch() // NOTE: branch savepoint only support local rollback + || tx.flags_.SHADOW_) { if (OB_NOT_NULL(extra_touched_ls)) { ret = OB_NOT_SUPPORTED; TRANS_LOG(WARN, "rollback on remote only suport collected tx parts", @@ -1030,7 +1063,7 @@ int ObTransService::rollback_to_local_implicit_savepoint_(ObTxDesc &tx, int64_t start_ts = ObTimeUtility::current_time(); ObTxRollbackParts rollback_parts; // when rollback local we use this from_scn to all downstream participants - ObTxSEQ from_scn = savepoint.clone_with_seq(ObSequence::inc_and_get_max_seq_no()); + ObTxSEQ from_scn = savepoint.clone_with_seq(ObSequence::inc_and_get_max_seq_no(), tx.seq_base_); if (OB_FAIL(find_parts_after_sp_(tx, parts, savepoint))) { TRANS_LOG(WARN, "find rollback parts fail", K(ret), K(savepoint), K(tx)); } else { @@ -1042,7 +1075,13 @@ int ObTransService::rollback_to_local_implicit_savepoint_(ObTxDesc &tx, TRANS_LOG(WARN, "get tx ctx fail", K(ret), K_(p.id), K(tx)); } else if (p.epoch_ != ctx->epoch_) { ret = OB_TRANS_CTX_NOT_EXIST; // FIXME more decent errno - } else if (OB_FAIL(ls_sync_rollback_savepoint__(ctx, savepoint, tx.op_sn_, expire_ts, from_scn, downstream_parts))) { + } else if (OB_FAIL(ls_sync_rollback_savepoint__(ctx, + savepoint, + tx.op_sn_, + tx.seq_base_, + expire_ts, + from_scn, + downstream_parts))) { TRANS_LOG(WARN, "LS rollback savepoint fail", K(ret), K(savepoint), K(tx)); } else { // merge find new downstream to tx.rollback parts @@ -1130,7 +1169,7 @@ int ObTransService::rollback_to_global_implicit_savepoint_(ObTxDesc &tx, * to ensure next retry can still recognize it is first stmt in transaction * we should reset tx state */ - reset_tx = tx_need_reset_(exec_errcode); + reset_tx = true; // tx_need_reset_(exec_errcode); reset_tx_state = !reset_tx; normal_rollback = !reset_tx; } else { @@ -1218,6 +1257,7 @@ int ObTransService::rollback_to_global_implicit_savepoint_(ObTxDesc &tx, int ObTransService::ls_sync_rollback_savepoint__(ObPartTransCtx *part_ctx, const ObTxSEQ savepoint, const int64_t op_sn, + const int64_t tx_seq_base, const int64_t expire_ts, const ObTxSEQ specified_from_scn, ObIArray &downstream_parts) @@ -1225,12 +1265,9 @@ int ObTransService::ls_sync_rollback_savepoint__(ObPartTransCtx *part_ctx, int ret = OB_SUCCESS; int64_t retry_cnt = 0; bool blockable = expire_ts > 0; - ObTxSEQ from_scn; - if (specified_from_scn.is_valid()) { - from_scn = specified_from_scn; - } else { - from_scn = savepoint.clone_with_seq(ObSequence::inc_and_get_max_seq_no()); - } + const int64_t seq_abs = ObSequence::inc_and_get_max_seq_no(); + const ObTxSEQ from_scn = specified_from_scn.is_valid() ? specified_from_scn + : savepoint.clone_with_seq(seq_abs, tx_seq_base); do { ret = part_ctx->rollback_to_savepoint(op_sn, from_scn, savepoint, downstream_parts); if (OB_NEED_RETRY == ret && blockable) { @@ -1456,6 +1493,7 @@ int ObTransService::rollback_savepoint_(ObTxDesc &tx, p.epoch_, tx.op_sn_, savepoint, + tx.seq_base_, born_epoch, &tx, false,/*for transfer*/ @@ -1525,6 +1563,7 @@ int ObTransService::ls_rollback_to_savepoint_(const ObTransID &tx_id, const int64_t verify_epoch, const int64_t op_sn, const ObTxSEQ savepoint, + const int64_t tx_seq_base, int64_t &ctx_born_epoch, const ObTxDesc *tx, const bool for_transfer, @@ -1580,7 +1619,13 @@ int ObTransService::ls_rollback_to_savepoint_(const ObTransID &tx_id, ret = OB_TRANS_CTX_NOT_EXIST; TRANS_LOG(WARN, "current ctx illegal, born epoch not match", K(ret), K(ls), K(tx_id), K(verify_epoch), KPC(ctx)); - } else if (OB_FAIL(ls_sync_rollback_savepoint__(ctx, savepoint, op_sn, expire_ts, from_scn, downstream_parts))) { + } else if(OB_FAIL(ls_sync_rollback_savepoint__(ctx, + savepoint, + op_sn, + tx_seq_base, + expire_ts, + from_scn, + downstream_parts))) { TRANS_LOG(WARN, "LS rollback to savepoint fail", K(ret), K(tx_id), K(ls), K(op_sn), K(savepoint), KPC(ctx)); } } @@ -1617,6 +1662,7 @@ inline int ObTransService::rollback_savepoint_slowpath_(ObTxDesc &tx, msg.tx_id_ = tx.tx_id_; msg.savepoint_ = savepoint; msg.op_sn_ = tx.op_sn_; + msg.tx_seq_base_ = tx.seq_base_; msg.epoch_ = -1; msg.request_id_ = tx_msg_id; msg.specified_from_scn_ = specified_from_scn; diff --git a/src/storage/tx/ob_tx_api.h b/src/storage/tx/ob_tx_api.h index a5f073f695..9fd5b34629 100644 --- a/src/storage/tx/ob_tx_api.h +++ b/src/storage/tx/ob_tx_api.h @@ -316,7 +316,30 @@ int create_implicit_savepoint(ObTxDesc &tx, const ObTxParam &tx_param, ObTxSEQ &savepoint, const bool release = false); - +/** + * create_branch_savepoint - establish a _branch_level_ savepoint and which can be + * used to rolling back to + * + * _branch_ level savepoint can accomplish partially rollback when multiple writer + * run in parallel at the same time window + * + * this was designed for internal use + * the savepoint won't be saved, for efficiency + * + * before you use this interface, a transaction must have been prepared either + * implicitly by `create_implicit_savepoint` or explicitly by `start_tx` + * + * @tx: the target transaction's descriptor + * transaction state if need + * @branch: the branch of the savepoint to establish + * @savepoint: the identifier of the savepoint returned + * + * Return: + * OB_SUCCESS - OK + */ +int create_branch_savepoint(ObTxDesc &tx, + const int16_t branch, + ObTxSEQ &savepoint); /** * create_explicit_savepoint - establish a savepoint and associate name * diff --git a/src/storage/tx/ob_tx_data_define.cpp b/src/storage/tx/ob_tx_data_define.cpp index da0bc607ee..afe14bda9a 100644 --- a/src/storage/tx/ob_tx_data_define.cpp +++ b/src/storage/tx/ob_tx_data_define.cpp @@ -478,8 +478,11 @@ int ObTxData::add_undo_action(ObTxTable *tx_table, transaction::ObUndoAction &ne } else if (OB_ISNULL(tx_data_table = tx_table->get_tx_data_table())) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "tx data table in tx table is nullptr.", KR(ret)); + } else if (OB_FAIL(merge_undo_actions_(tx_data_table, node, new_undo_action))) { + STORAGE_LOG(WARN, "merge undo actions fail.", KR(ret), K(new_undo_action)); + } else if (!new_undo_action.is_valid()) { + // if new_undo_action is merged, it will be set to invalid and skip insert } else { - merge_undo_actions_(tx_data_table, node, new_undo_action); // generate new node if current node cannot be inserted if (OB_ISNULL(node) || node->size_ >= TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE) { ObUndoStatusNode *new_node = nullptr; @@ -521,10 +524,12 @@ int ObTxData::merge_undo_actions_(ObTxDataTable *tx_data_table, int ret = OB_SUCCESS; while (OB_SUCC(ret) && OB_NOT_NULL(node)) { for (int i = node->size_ - 1; i >= 0; i--) { - if (new_undo_action.is_contain(node->undo_actions_[i]) - || node->undo_actions_[i].is_contain(new_undo_action)) { - new_undo_action.merge(node->undo_actions_[i]); - node->size_--; + if (new_undo_action.is_contain(node->undo_actions_[i])) { + node->size_--; // pop merged + } else if (node->undo_actions_[i].is_contain(new_undo_action)) { + // new undo is merged, reset it + new_undo_action.reset(); + break; } else { break; } @@ -553,7 +558,6 @@ int ObTxData::merge_undo_actions_(ObTxDataTable *tx_data_table, return ret; } - bool ObTxData::equals_(ObTxData &rhs) { diff --git a/src/storage/tx/ob_tx_free_route_state.cpp b/src/storage/tx/ob_tx_free_route_state.cpp index d4f50a5830..c920eb7676 100644 --- a/src/storage/tx/ob_tx_free_route_state.cpp +++ b/src/storage/tx/ob_tx_free_route_state.cpp @@ -115,7 +115,8 @@ TXN_FREE_ROUTE_MEMBERS(static, , PRE_STATIC_DECODE, POST_STATIC_DECODE, access_mode_, sess_id_, timeout_us_, - expire_ts_); + expire_ts_, + seq_base_); TXN_FREE_ROUTE_MEMBERS(dynamic, PRE_ENCODE_DYNAMIC_FOR_VERIFY, PRE_DYNAMIC_DECODE, POST_DYNAMIC_DECODE, op_sn_, state_, @@ -136,7 +137,8 @@ TXN_FREE_ROUTE_MEMBERS(extra, , PRE_EXTRA_DECODE, POST_EXTRA_DECODE, addr_, // dup with static isolation_, // dup with static snapshot_version_, - snapshot_scn_); + snapshot_scn_, + seq_base_); #undef TXN_FREE_ROUTE_MEMBERS int64_t ObTxDesc::estimate_state_size() diff --git a/src/storage/tx/ob_tx_log.cpp b/src/storage/tx/ob_tx_log.cpp index 127131a7a1..89636412b1 100644 --- a/src/storage/tx/ob_tx_log.cpp +++ b/src/storage/tx/ob_tx_log.cpp @@ -155,6 +155,9 @@ int ObCtxRedoInfo::before_serialize() if (compat_bytes_.is_inited()) { if (OB_FAIL(compat_bytes_.set_all_member_need_ser())) { TRANS_LOG(WARN, "reset all compat_bytes_ valid failed", K(ret)); + } else { + // skip serialize cluster_version, since 4.3, cluster_version put in LogBlockHeader + TX_NO_NEED_SER(cluster_version_ >= DATA_VERSION_4_3_0_0, 1, compat_bytes_); } } else { if (OB_FAIL(compat_bytes_.init(1))) { @@ -253,7 +256,8 @@ OB_TX_SERIALIZE_MEMBER(ObTxActiveInfoLog, /* 15 */ last_seq_no_, /* 16 */ cluster_version_, /* 17 */ max_submitted_seq_no_, - /* 18 */ xid_); + /* 18 */ xid_, + /* 19 */ serial_final_seq_no_); OB_TX_SERIALIZE_MEMBER(ObTxCommitInfoLog, compat_bytes_, @@ -287,7 +291,8 @@ OB_TX_SERIALIZE_MEMBER(ObTxCommitLog, /* 5 */ trans_type_, /* 6 */ tx_data_backup_, /* 7 */ prev_lsn_, - /* 8 */ ls_log_info_arr_); + /* 8 */ ls_log_info_arr_, + /* 9 */ checksum_sig_serde_); OB_TX_SERIALIZE_MEMBER(ObTxClearLog, compat_bytes_, /* 1 */ incremental_participants_); @@ -313,7 +318,7 @@ int ObTxActiveInfoLog::before_serialize() TRANS_LOG(WARN, "reset all compat_bytes_ valid failed", K(ret)); } } else { - if (OB_FAIL(compat_bytes_.init(18))) { + if (OB_FAIL(compat_bytes_.init(19))) { TRANS_LOG(WARN, "init compat_bytes_ failed", K(ret)); } } @@ -334,9 +339,10 @@ int ObTxActiveInfoLog::before_serialize() TX_NO_NEED_SER(last_op_sn_ == 0, 13, compat_bytes_); TX_NO_NEED_SER(!first_seq_no_.is_valid(), 14, compat_bytes_); TX_NO_NEED_SER(!last_seq_no_.is_valid(), 15, compat_bytes_); - TX_NO_NEED_SER(cluster_version_ == 0, 16, compat_bytes_); + TX_NO_NEED_SER((cluster_version_ == 0 || cluster_version_ >= DATA_VERSION_4_3_0_0), 16, compat_bytes_); TX_NO_NEED_SER(!max_submitted_seq_no_.is_valid(), 17, compat_bytes_); TX_NO_NEED_SER(xid_.empty(), 18, compat_bytes_); + TX_NO_NEED_SER(!serial_final_seq_no_.is_valid(), 19, compat_bytes_); } return ret; @@ -364,7 +370,7 @@ int ObTxCommitInfoLog::before_serialize() TX_NO_NEED_SER(is_dup_tx_ == false, 5, compat_bytes_); TX_NO_NEED_SER(can_elr_ == false, 6, compat_bytes_); TX_NO_NEED_SER(incremental_participants_.empty(), 7, compat_bytes_); - TX_NO_NEED_SER(cluster_version_ == 0, 8, compat_bytes_); + TX_NO_NEED_SER((cluster_version_ == 0 || cluster_version_ >= DATA_VERSION_4_3_0_0), 8, compat_bytes_); TX_NO_NEED_SER(app_trace_id_str_.empty(), 9, compat_bytes_); TX_NO_NEED_SER(app_trace_info_.empty(), 10, compat_bytes_); TX_NO_NEED_SER(prev_record_lsn_.is_valid() == false, 11, compat_bytes_); @@ -407,7 +413,7 @@ int ObTxCommitLog::before_serialize() TRANS_LOG(WARN, "reset all compat_bytes_ valid failed", K(ret)); } } else { - if (OB_FAIL(compat_bytes_.init(8))) { + if (OB_FAIL(compat_bytes_.init(9))) { TRANS_LOG(WARN, "init compat_bytes_ failed", K(ret)); } } @@ -421,6 +427,7 @@ int ObTxCommitLog::before_serialize() TX_NO_NEED_SER(false, 6, compat_bytes_); // tx_data_backup_ TX_NO_NEED_SER(prev_lsn_.is_valid() == false, 7, compat_bytes_); TX_NO_NEED_SER(ls_log_info_arr_.empty(), 8, compat_bytes_); + TX_NO_NEED_SER(checksum_sig_.count() == 0, 9, compat_bytes_); } return ret; } @@ -590,6 +597,7 @@ int ObTxRedoLog::set_mutator_size(const int64_t size, const bool after_fill) } else if (!after_fill) { int len = 0; SERIALIZE_SIZE_HEADER(UNIS_VERSION); + ret = ctx_redo_info_.before_serialize(); len = len + MUTATOR_SIZE_NEED_BYTES + ctx_redo_info_.get_serialize_size(); if (size <= len) { ret = OB_SIZE_OVERFLOW; @@ -777,7 +785,7 @@ int ObTxRedoLog::format_mutator_row_(const memtable::ObMemtableMutatorRow &row, arg.writer_ptr_->dump_key("Flag"); arg.writer_ptr_->dump_int64(flag); arg.writer_ptr_->dump_key("SeqNo"); - arg.writer_ptr_->dump_int64(seq_no.cast_to_int()); + arg.writer_ptr_->dump_string(to_cstring(seq_no)); arg.writer_ptr_->dump_key("NewRowSize"); arg.writer_ptr_->dump_int64(new_row.size_); arg.writer_ptr_->dump_key("OldRowSize"); @@ -1063,13 +1071,60 @@ int ObTxAbortLog::init_tx_data_backup(const share::SCN &start_scn) } // ============================== Tx Log Blcok ============================= +#define TX_LOG_BLOCK_SERIALIZE_MEMBERS \ + org_cluster_id_, \ + log_entry_no_, \ + tx_id_, \ + scheduler_, \ + __log_entry_no_, \ + cluster_version_, \ + flags_ -OB_TX_SERIALIZE_MEMBER(ObTxLogBlockHeader, - compat_bytes_, - org_cluster_id_, - log_entry_no_, - tx_id_, - scheduler_); +OB_DEF_SERIALIZE(ObTxLogBlockHeader) +{ + int64_t pos_bk = pos; + int ret = OB_SUCCESS; + TX_SER_COMPAT_BYTES(compat_bytes_); + TX_LST_DO_CODE(OB_TX_UNIS_ENCODE, compat_bytes_, TX_LOG_BLOCK_SERIALIZE_MEMBERS); + // if the real serialized data size less than reserved + // must do pading + const int64_t pading_size = serialize_size_ - (pos - pos_bk); + MEMSET(buf + pos, 0 , pading_size); + pos += pading_size; + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObTxLogBlockHeader) +{ + if (serialize_size_ == 0) { + ob_abort(); + } + return serialize_size_; +} + +void ObTxLogBlockHeader::calc_serialize_size_() +{ + int64_t len = 0; + int64_t log_entry_no = log_entry_no_; + int64_t log_entry_no_ = INT64_MAX; + uint8_t flags = flags_; + uint8_t flags_ = UINT8_MAX; + TX_SER_SIZE_COMPAT_BYTES(compat_bytes_); + TX_LST_DO_CODE(OB_TX_UNIS_ADD_LEN, compat_bytes_, TX_LOG_BLOCK_SERIALIZE_MEMBERS); + flags_ = flags; + log_entry_no_ = log_entry_no; + serialize_size_ = len; +} + +OB_DEF_DESERIALIZE(ObTxLogBlockHeader) +{ + int ret = OB_SUCCESS; + UNF_UNUSED_DES; + TX_DSER_COMPAT_BYTES(compat_bytes_); + TX_LST_DO_CODE(OB_TX_UNIS_DECODE, compat_bytes_, TX_LOG_BLOCK_SERIALIZE_MEMBERS); + serialize_size_ = data_len; + return ret; +} int ObTxLogBlockHeader::before_serialize() { @@ -1078,9 +1133,19 @@ int ObTxLogBlockHeader::before_serialize() if (compat_bytes_.is_inited()) { if (OB_FAIL(compat_bytes_.set_all_member_need_ser())) { TRANS_LOG(WARN, "reset all compat_bytes_ valid failed", K(ret)); + } else { + if (cluster_version_ == 0) { + ob_abort(); + } + if (cluster_version_ >= DATA_VERSION_4_3_0_0) { + TX_NO_NEED_SER(true, 2, compat_bytes_); + } + if (serialize_size_ == 0) { + calc_serialize_size_(); + } } } else { - if (OB_FAIL(compat_bytes_.init(4))) { + if (OB_FAIL(compat_bytes_.init(7))) { TRANS_LOG(WARN, "init compat_bytes_ failed", K(ret)); } } @@ -1096,67 +1161,73 @@ const int64_t ObTxLogBlock::BIG_SEGMENT_SPILT_SIZE = common::OB_MAX_LOG_ALLOWED_ void ObTxLogBlock::reset() { + inited_ = false; + log_base_header_.reset(); + header_.reset(); fill_buf_.reset(); replay_buf_ = nullptr; len_ = pos_ = 0; cur_log_type_ = ObTxLogType::UNKNOWN; - cur_block_barrier_type_ = logservice::ObReplayBarrierType::NO_NEED_BARRIER; cb_arg_array_.reset(); big_segment_buf_ = nullptr; } -int ObTxLogBlock::reuse(const int64_t replay_hint, const ObTxLogBlockHeader &block_header) +int ObTxLogBlock::reuse_for_fill() { int ret = OB_SUCCESS; - cur_log_type_ = ObTxLogType::UNKNOWN; - cur_block_barrier_type_ = logservice::ObReplayBarrierType::NO_NEED_BARRIER; - cb_arg_array_.reset(); - big_segment_buf_ = nullptr; - pos_ = 0; - if (OB_FAIL(serialize_log_block_header_(replay_hint, block_header))) { - TRANS_LOG(ERROR, "serialize log block header error when reuse", K(ret), KPC(this)); + if (!header_.is_valid()) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "must init header before reuse", K(ret)); + } else { + log_base_header_.reset(); + cur_log_type_ = ObTxLogType::UNKNOWN; + cb_arg_array_.reset(); + big_segment_buf_ = nullptr; + pos_ = 0; + // reserve place for headers, header will be filled back + pos_ += log_base_header_.get_serialize_size(); // assume FIXED size + header_.before_serialize(); + pos_ += header_.get_serialize_size(); } return ret; } ObTxLogBlock::ObTxLogBlock() - : replay_buf_(nullptr), len_(0), pos_(0), cur_log_type_(ObTxLogType::UNKNOWN), - cur_block_barrier_type_(logservice::ObReplayBarrierType::NO_NEED_BARRIER), cb_arg_array_(), + : inited_(false), log_base_header_(), header_(), + replay_buf_(nullptr), len_(0), pos_(0), cur_log_type_(ObTxLogType::UNKNOWN), cb_arg_array_(), big_segment_buf_(nullptr) { // do nothing } -int ObTxLogBlock::init(const int64_t replay_hint, - const ObTxLogBlockHeader &block_header, - const int64_t suggested_buf_size) +int ObTxLogBlock::init_for_fill(const int64_t suggested_buf_size) { int ret = OB_SUCCESS; // accept the suggested buffer size const int64_t buf_size = suggested_buf_size; - if (OB_NOT_NULL(replay_buf_) || !block_header.is_valid()) { + if (OB_NOT_NULL(replay_buf_) || !header_.is_valid()) { ret = OB_INVALID_ARGUMENT; - TRANS_LOG(ERROR, "invalid argument", K(replay_hint), K(*this), K(block_header)); + TRANS_LOG(ERROR, "invalid argument", K(*this), K_(header)); } else if (OB_FAIL(fill_buf_.init(buf_size))) { - TRANS_LOG(WARN, "fill log buffer init error", K(ret), K(replay_hint), K(block_header), K(buf_size)); + TRANS_LOG(WARN, "fill log buffer init error", K(ret), K(buf_size)); } else { len_ = fill_buf_.get_length(); pos_ = 0; - if (OB_FAIL(serialize_log_block_header_(replay_hint, block_header))) { - ret = OB_SERIALIZE_ERROR; - TRANS_LOG(WARN, "serialize log block header error", K(ret), K(*this)); - } + // reserve place for headers, header will be filled back + pos_ += log_base_header_.get_serialize_size(); // assume FIXED size + header_.before_serialize(); + pos_ += header_.get_serialize_size(); + inited_ = true; } return ret; } -int ObTxLogBlock::init_with_header(const char *buf, - const int64_t &size, - int64_t &replay_hint, - ObTxLogBlockHeader &block_header) +int ObTxLogBlock::init_for_replay(const char *buf, const int64_t &size) { int ret = OB_SUCCESS; - if (OB_NOT_NULL(replay_buf_) + if (inited_) { + ret = OB_INIT_TWICE; + } else if (OB_NOT_NULL(replay_buf_) || OB_ISNULL(buf) || size <= 0) { ret = OB_INVALID_ARGUMENT; @@ -1165,18 +1236,22 @@ int ObTxLogBlock::init_with_header(const char *buf, replay_buf_ = buf; len_ = size; pos_ = 0; - if (OB_FAIL(deserialize_log_block_header_(replay_hint, block_header))) { + if (OB_FAIL(deserialize_log_block_header_())) { ret = OB_DESERIALIZE_ERROR; TRANS_LOG(WARN, "deserialize log block header error", K(ret), K(*this)); + } else { + inited_ = true; } } return ret; } -int ObTxLogBlock::init(const char *buf, const int64_t &size, int skip_pos, ObTxLogBlockHeader &block_header) +int ObTxLogBlock::init_for_replay(const char *buf, const int64_t &size, int skip_pos) { int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || size <= 0) { + if (inited_) { + ret = OB_INIT_TWICE; + } else if (OB_ISNULL(buf) || size <= 0) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(ERROR, "invalid argument", K(buf), K(size), K(*this)); } else { @@ -1184,57 +1259,27 @@ int ObTxLogBlock::init(const char *buf, const int64_t &size, int skip_pos, ObTxL len_ = size; pos_ = skip_pos; - if (OB_FAIL(block_header.deserialize(replay_buf_, len_, pos_))) { + if (OB_FAIL(header_.deserialize(replay_buf_, len_, pos_))) { TRANS_LOG(WARN, "deserialize block header", K(ret)); + } else { + inited_ = true; } } return ret; } -int ObTxLogBlock::rewrite_barrier_log_block(int64_t replay_hint, - const enum logservice::ObReplayBarrierType barrier_type) +int ObTxLogBlock::seal(const int64_t replay_hint, const ObReplayBarrierType barrier_type) { int ret = OB_SUCCESS; - int64_t tmp_pos = 0; - char *serialize_buf = nullptr; - logservice::ObReplayBarrierType final_barrier_type = - logservice::ObReplayBarrierType::NO_NEED_BARRIER; - - if (OB_ISNULL(fill_buf_.get_buf()) - || logservice::ObReplayBarrierType::INVALID_BARRIER == barrier_type) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid arguments", K(ret), K(replay_hint), K(barrier_type), KPC(this)); - } else if (OB_FAIL(ObTxLogTypeChecker::decide_final_barrier_type(cur_block_barrier_type_, - final_barrier_type))) { - TRANS_LOG(WARN, "decide final barrier type with the cur_block_barrier failed", K(ret), - K(barrier_type), K(final_barrier_type), K(replay_hint)); - } else if (OB_FAIL( - ObTxLogTypeChecker::decide_final_barrier_type(barrier_type, final_barrier_type))) { - TRANS_LOG(WARN, "decide final barrier type with the barrier_type arg failed", K(ret), - K(barrier_type), K(final_barrier_type), K(replay_hint)); + log_base_header_ = logservice::ObLogBaseHeader(logservice::ObLogBaseType::TRANS_SERVICE_LOG_BASE_TYPE, + barrier_type, replay_hint); + int64_t pos_bk = pos_; + pos_ = 0; + if (OB_FAIL(serialize_log_block_header_())) { + TRANS_LOG(WARN, "serialize log block header error", K(ret)); } else { - serialize_buf = fill_buf_.get_buf(); + pos_ = pos_bk; } - - if (OB_FAIL(ret)) { - // do nothing - } else if (OB_ISNULL(serialize_buf)) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "unexpected empty serialize_buf", K(*this)); - } else { - logservice::ObLogBaseHeader header(logservice::ObLogBaseType::TRANS_SERVICE_LOG_BASE_TYPE, - final_barrier_type, replay_hint); - - if (final_barrier_type != barrier_type) { - TRANS_LOG(INFO, "rewrite barrier_type without the origin target", K(ret), K(replay_hint), - K(final_barrier_type), K(barrier_type), KPC(this)); - } - - if (OB_FAIL(header.serialize(serialize_buf, len_, tmp_pos))) { - TRANS_LOG(WARN, "serialize log base header error", K(ret)); - } - } - return ret; } @@ -1251,11 +1296,7 @@ int ObTxLogBlock::set_prev_big_segment_scn(const share::SCN prev_scn) return ret; } -int ObTxLogBlock::acquire_segment_log_buf(const char *&submit_buf, - int64_t &submit_buf_len, - const ObTxLogBlockHeader &block_header, - const ObTxLogType big_segment_log_type, - ObTxBigSegmentBuf *big_segment_buf) +int ObTxLogBlock::acquire_segment_log_buf(const ObTxLogType big_segment_log_type, ObTxBigSegmentBuf *big_segment_buf) { int ret = OB_SUCCESS; bool need_fill_part_scn = false; @@ -1272,8 +1313,8 @@ int ObTxLogBlock::acquire_segment_log_buf(const char *&submit_buf, ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, " big segment_buf", K(ret), KPC(this)); } else if (OB_FALSE_IT(tmp_segment_buf = big_segment_buf_)) { - } else if (OB_FAIL(reuse(block_header.get_tx_id(), block_header))) { - TRANS_LOG(WARN, "serialize log block header failed", K(ret), K(block_header), KPC(this)); + } else if (OB_FAIL(reuse_for_fill())) { + TRANS_LOG(WARN, "reuse log block failed", K(ret), KPC(this)); } else if (OB_FAIL(log_type_header.serialize(fill_buf_.get_buf(), len_, pos_))) { TRANS_LOG(WARN, "serialize log type header failed", K(ret), KPC(this)); } else if (OB_FAIL(cb_arg_array_.push_back(ObTxCbArg(ObTxLogType::TX_BIG_SEGMENT_LOG, NULL)))) { @@ -1282,9 +1323,7 @@ int ObTxLogBlock::acquire_segment_log_buf(const char *&submit_buf, TRANS_LOG(WARN, "push the second log type arg failed", K(ret), K(*this)); } else if (OB_FAIL(tmp_segment_buf->split_one_part(fill_buf_.get_buf(), BIG_SEGMENT_SPILT_SIZE, pos_, need_fill_part_scn))) { - TRANS_LOG(WARN, "acquire a part of big segment failed", K(ret), K(block_header), KPC(this)); - } else if (OB_FALSE_IT(submit_buf = fill_buf_.get_buf())) { - } else if (OB_FALSE_IT(submit_buf_len = pos_)) { + TRANS_LOG(WARN, "acquire a part of big segment failed", K(ret), KPC(this)); } else if (tmp_segment_buf->is_completed()) { // tmp_segment_buf->reset(); // reset big_segment buf after set prev scn @@ -1298,14 +1337,10 @@ int ObTxLogBlock::acquire_segment_log_buf(const char *&submit_buf, return ret; } -int ObTxLogBlock::serialize_log_block_header_(const int64_t replay_hint, - const ObTxLogBlockHeader &block_header, - const logservice::ObReplayBarrierType barrier_type) +int ObTxLogBlock::serialize_log_block_header_() { int ret = OB_SUCCESS; char *serialize_buf = nullptr; - logservice::ObLogBaseHeader header(logservice::ObLogBaseType::TRANS_SERVICE_LOG_BASE_TYPE, - barrier_type, replay_hint); if (OB_ISNULL(fill_buf_.get_buf()) || pos_ != 0) { ret = OB_INVALID_ARGUMENT; } else { @@ -1317,31 +1352,26 @@ int ObTxLogBlock::serialize_log_block_header_(const int64_t replay_hint, } else if (OB_ISNULL(serialize_buf)) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "unexpected empty serialize_buf", K(*this)); - } else if (OB_FAIL(header.serialize(serialize_buf, len_, pos_))) { + } else if (OB_FAIL(log_base_header_.serialize(serialize_buf, len_, pos_))) { TRANS_LOG(WARN, "serialize log base header error", K(ret)); - } else if (OB_FAIL(block_header.serialize(serialize_buf, len_, pos_))) { + } else if (OB_FAIL(header_.before_serialize())) { + TRANS_LOG(WARN, "before serialize failed", K(ret), K(*this)); + } else if (OB_FAIL(header_.serialize(serialize_buf, len_, pos_))) { TRANS_LOG(WARN, "serialize block header error", K(ret)); } - if (OB_SUCC(ret)) { - cur_block_barrier_type_ = barrier_type; - } - return ret; } -int ObTxLogBlock::deserialize_log_block_header_(int64_t &replay_hint, ObTxLogBlockHeader &block_header) +int ObTxLogBlock::deserialize_log_block_header_() { int ret = OB_SUCCESS; - logservice::ObLogBaseHeader header; if (OB_ISNULL(replay_buf_) || pos_ != 0) { ret = OB_INVALID_ARGUMENT; - } else if (OB_FAIL(header.deserialize(replay_buf_, len_, pos_))) { + } else if (OB_FAIL(log_base_header_.deserialize(replay_buf_, len_, pos_))) { TRANS_LOG(WARN, "deserialize log base header error", K(ret),K(len_),K(pos_)); - } else if (OB_FAIL(block_header.deserialize(replay_buf_, len_, pos_))) { + } else if (OB_FAIL(header_.deserialize(replay_buf_, len_, pos_))) { TRANS_LOG(WARN, "deserialize block header", K(ret), K(len_), K(pos_)); - } else { - replay_hint = header.get_replay_hint(); } return ret; } @@ -1410,7 +1440,7 @@ int ObTxLogBlock::get_next_log(ObTxLogHeader &header, } } } - // TRANS_LOG(INFO, "[TxLogBlock] get_next_log in replay",K(cur_log_type_), K(len_), K(pos_)); + TRANS_LOG(DEBUG, "[TxLogBlock] get_next_log in replay",K(cur_log_type_), K(len_), K(pos_)); } return ret; } diff --git a/src/storage/tx/ob_tx_log.h b/src/storage/tx/ob_tx_log.h index 958550561f..2d1fe6f7df 100644 --- a/src/storage/tx/ob_tx_log.h +++ b/src/storage/tx/ob_tx_log.h @@ -26,6 +26,7 @@ #include "logservice/ob_log_base_header.h" #include "logservice/palf/lsn.h" #include "share/scn.h" +#include "lib/utility/ob_unify_serialize.h" //#include #define OB_TX_MDS_LOG_USE_BIT_SEGMENT_BUF @@ -246,7 +247,7 @@ public: { before_serialize(); } - ObTxRedoLog(const int64_t &log_no, const uint64_t &cluster_version) + ObTxRedoLog(const uint64_t &cluster_version) : mutator_buf_(nullptr), replay_mutator_buf_(nullptr), mutator_size_(-1), ctx_redo_info_(cluster_version) // (ctx_redo_info_.clog_encrypt_info_)(encrypt_info),(ctx_redo_info_.cluster_version_)(cluster_version) @@ -299,8 +300,6 @@ private: int64_t mutator_size_; ObCtxRedoInfo ctx_redo_info_; - //--------- for liboblog ----------- - // int64_t log_no_; }; // for dist trans write it's multi source data, the same as redo, @@ -359,7 +358,8 @@ public: app_trace_id_str_(temp_ref.app_trace_id_str_), schema_version_(0), can_elr_(false), proposal_leader_(temp_ref.proposal_leader_), cur_query_start_time_(0), is_sub2pc_(false), is_dup_tx_(false), tx_expired_time_(0), epoch_(0), last_op_sn_(0), first_seq_no_(), - last_seq_no_(), max_submitted_seq_no_(), cluster_version_(0), xid_(temp_ref.xid_) + last_seq_no_(), max_submitted_seq_no_(), serial_final_seq_no_(), cluster_version_(0), + xid_(temp_ref.xid_) { before_serialize(); } @@ -380,14 +380,15 @@ public: ObTxSEQ last_seq_no, ObTxSEQ max_submitted_seq_no, uint64_t cluster_version, - const ObXATransID &xid) + const ObXATransID &xid, + ObTxSEQ serial_final_seq_no) : scheduler_(scheduler), trans_type_(trans_type), session_id_(session_id), app_trace_id_str_(app_trace_id_str), schema_version_(schema_version), can_elr_(elr), proposal_leader_(proposal_leader), cur_query_start_time_(cur_query_start_time), is_sub2pc_(is_sub2pc), is_dup_tx_(is_dup_tx), tx_expired_time_(tx_expired_time), epoch_(epoch), last_op_sn_(last_op_sn), first_seq_no_(first_seq_no), last_seq_no_(last_seq_no), - max_submitted_seq_no_(max_submitted_seq_no), cluster_version_(cluster_version), - xid_(xid) + max_submitted_seq_no_(max_submitted_seq_no), serial_final_seq_no_(serial_final_seq_no), + cluster_version_(cluster_version), xid_(xid) { before_serialize(); }; @@ -408,6 +409,7 @@ public: ObTxSEQ get_first_seq_no() const { return first_seq_no_; } ObTxSEQ get_last_seq_no() const { return last_seq_no_; } ObTxSEQ get_max_submitted_seq_no() const { return max_submitted_seq_no_; } + ObTxSEQ get_serial_final_seq_no() const { return serial_final_seq_no_; } uint64_t get_cluster_version() const { return cluster_version_; } const ObXATransID &get_xid() const { return xid_; } // for ob_admin @@ -431,6 +433,7 @@ public: K(first_seq_no_), K(last_seq_no_), K(max_submitted_seq_no_), + K(serial_final_seq_no_), K(cluster_version_), K(xid_)); @@ -459,7 +462,7 @@ private: // ctrl savepoint written log ObTxSEQ max_submitted_seq_no_; - + ObTxSEQ serial_final_seq_no_; uint64_t cluster_version_; ObXATransID xid_; }; @@ -644,9 +647,11 @@ private: class ObTxCommitLogTempRef { public: - ObTxCommitLogTempRef() : incremental_participants_(), multi_source_data_(), ls_log_info_arr_() {} + ObTxCommitLogTempRef() : checksum_signature_(), incremental_participants_(), multi_source_data_(), ls_log_info_arr_() + { checksum_signature_.set_max_print_count(1024); } public: + ObSEArray checksum_signature_; share::ObLSArray incremental_participants_; ObTxBufferNodeArray multi_source_data_; ObLSLogInfoArray ls_log_info_arr_; @@ -661,7 +666,8 @@ public: public: ObTxCommitLog(ObTxCommitLogTempRef &temp_ref) - : commit_version_(), checksum_(0), + : commit_version_(), checksum_(0), checksum_sig_(temp_ref.checksum_signature_), + checksum_sig_serde_(checksum_sig_), incremental_participants_(temp_ref.incremental_participants_), multi_source_data_(temp_ref.multi_source_data_), trans_type_(TransType::SP_TRANS), tx_data_backup_(), prev_lsn_(), ls_log_info_arr_(temp_ref.ls_log_info_arr_) @@ -670,12 +676,13 @@ public: } ObTxCommitLog(share::SCN commit_version, uint64_t checksum, + ObIArray &checksum_sig, share::ObLSArray &incremental_participants, ObTxBufferNodeArray &multi_source_data, int32_t trans_type, LogOffSet prev_lsn, ObLSLogInfoArray &ls_log_info_arr) - : checksum_(checksum), + : checksum_(checksum), checksum_sig_(checksum_sig), checksum_sig_serde_(checksum_sig_), incremental_participants_(incremental_participants), multi_source_data_(multi_source_data), trans_type_(trans_type), prev_lsn_(prev_lsn), ls_log_info_arr_(ls_log_info_arr) { @@ -701,6 +708,7 @@ public: TO_STRING_KV(K(LOG_TYPE), K(commit_version_), K(checksum_), + K_(checksum_sig), K(incremental_participants_), K(multi_source_data_), K(trans_type_), @@ -715,6 +723,8 @@ private: ObTxSerCompatByte compat_bytes_; share::SCN commit_version_; // equal to INVALID_COMMIT_VERSION in Single LS Transaction uint64_t checksum_; + ObIArray &checksum_sig_; + ObIArraySerDeTrait checksum_sig_serde_; share::ObLSArray &incremental_participants_; ObTxBufferNodeArray &multi_source_data_; @@ -752,7 +762,6 @@ public: { before_serialize(); } - const share::ObLSArray &get_incremental_participants() const { return incremental_participants_; } int ob_admin_dump(share::ObAdminMutatorStringArg &arg); @@ -922,48 +931,81 @@ private: class ObTxLogBlockHeader { OB_UNIS_VERSION(1); - +public: + struct FixSizeTrait_int64_t { + FixSizeTrait_int64_t(int64_t &v): v_(v) {} + int64_t &v_; + int serialize(SERIAL_PARAMS) const { return NS_::encode_i64(buf, buf_len, pos, v_); } + int64_t get_serialize_size(void) const { return NS_::encoded_length_i64(v_); } + int deserialize(DESERIAL_PARAMS) { return NS_::decode_i64(buf, data_len, pos, &v_); } + }; public: void reset() { org_cluster_id_ = 0; + cluster_version_ = 0; log_entry_no_ = 0; tx_id_ = 0; scheduler_.reset(); + flags_ = 0; + serialize_size_ = 0; } - ObTxLogBlockHeader() + ObTxLogBlockHeader(): __log_entry_no_(log_entry_no_) { reset(); before_serialize(); }; ObTxLogBlockHeader(const uint64_t org_cluster_id, + const int64_t cluster_version, const int64_t log_entry_no, const ObTransID &tx_id, const common::ObAddr &scheduler) - : org_cluster_id_(org_cluster_id), log_entry_no_(log_entry_no), tx_id_(tx_id), - scheduler_(scheduler) + : __log_entry_no_(log_entry_no_), flags_(0) { + init(org_cluster_id, cluster_version, log_entry_no, tx_id, scheduler); before_serialize(); } - + void init(const uint64_t org_cluster_id, + const int64_t cluster_version, + const int64_t log_entry_no, + const ObTransID &tx_id, + const common::ObAddr &scheduler) { + org_cluster_id_ = org_cluster_id; + cluster_version_ = cluster_version; + log_entry_no_ = log_entry_no; + tx_id_ = tx_id; + scheduler_ = scheduler; + flags_ = 0; + serialize_size_ = 0; + } + void calc_serialize_size_(); uint64_t get_org_cluster_id() const { return org_cluster_id_; } + int64_t get_cluster_version() const { return cluster_version_; } int64_t get_log_entry_no() const { return log_entry_no_; } + void set_log_entry_no(int64_t entry_no) { log_entry_no_ = entry_no; } const ObTransID &get_tx_id() const { return tx_id_; } const common::ObAddr &get_scheduler() const { return scheduler_; } bool is_valid() const { return org_cluster_id_ >= 0; } - - TO_STRING_KV(K_(org_cluster_id), K_(log_entry_no), K_(tx_id), K_(scheduler)); + void set_serial_final() { flags_ |= SERIAL_FINAL; } + bool is_serial_final() const { return (flags_ & SERIAL_FINAL) == SERIAL_FINAL; } + uint8_t flags() const { return flags_; } + TO_STRING_KV(K_(compat_bytes), K_(org_cluster_id), K_(cluster_version), K_(log_entry_no), K_(tx_id), K_(scheduler), K_(flags)); public: int before_serialize(); - + // the last serial log + static const uint8_t SERIAL_FINAL = ((uint8_t)1) << 0; private: + int64_t serialize_size_; ObTxSerCompatByte compat_bytes_; uint64_t org_cluster_id_; + int64_t cluster_version_; int64_t log_entry_no_; + FixSizeTrait_int64_t __log_entry_no_; // serialize helper member, hiden for others ObTransID tx_id_; common::ObAddr scheduler_; + uint8_t flags_; }; class ObTxAdaptiveLogBuf @@ -1096,23 +1138,19 @@ public: static const logservice::ObLogBaseType DEFAULT_LOG_BLOCK_TYPE; // TRANS_LOG static const int32_t DEFAULT_BIG_ROW_BLOCK_SIZE; static const int64_t BIG_SEGMENT_SPILT_SIZE; - NEED_SERIALIZE_AND_DESERIALIZE; ObTxLogBlock(); void reset(); - int reuse(const int64_t replay_hint, const ObTxLogBlockHeader &block_header); - int init(const int64_t replay_hint, - const ObTxLogBlockHeader &block_header, - const int64_t suggested_buf_size = ObTxAdaptiveLogBuf::NORMAL_LOG_BUF_SIZE); - int init_with_header(const char *buf, - const int64_t &size, - int64_t &replay_hint, - ObTxLogBlockHeader &block_header); - int init(const char *buf, - const int64_t &size, - int skip_pos, - ObTxLogBlockHeader &block_header); // init before replay + int reuse_for_fill(); + int init_for_fill(const int64_t suggested_buf_size = ObTxAdaptiveLogBuf::NORMAL_LOG_BUF_SIZE); + int init_for_replay(const char *buf, const int64_t &size); + int init_for_replay(const char *buf, const int64_t &size, int skip_pos); // init before replay + ObTxLogBlockHeader &get_header() { return header_; } + logservice::ObLogBaseHeader &get_log_base_header() { return log_base_header_; } + typedef logservice::ObReplayBarrierType ObReplayBarrierType; + int seal(const int64_t replay_hint, const ObReplayBarrierType barrier_type = ObReplayBarrierType::NO_NEED_BARRIER); ~ObTxLogBlock() { reset(); } + bool is_inited() const { return inited_; } int get_next_log(ObTxLogHeader &header, ObTxBigSegmentBuf *big_segment_buf = nullptr, bool *contain_big_segment = nullptr); @@ -1126,19 +1164,16 @@ public: int prepare_mutator_buf(ObTxRedoLog &redo); int finish_mutator_buf(ObTxRedoLog &redo, const int64_t &mutator_size); int extend_log_buf(); - - //rewrite base log header at the head of log block. - //It is a temporary interface for create/remove tablet which can not be used for any other target - int rewrite_barrier_log_block(int64_t replay_hint, - const enum logservice::ObReplayBarrierType barrier_type); TO_STRING_KV(K(fill_buf_), KP(replay_buf_), K(len_), K(pos_), K(cur_log_type_), - K(cur_block_barrier_type_), K(cb_arg_array_), - KPC(big_segment_buf_)); + KPC(big_segment_buf_), + K_(inited), + K_(header), + K_(log_base_header)); public: // get fill buf for submit log @@ -1146,27 +1181,21 @@ public: const int64_t &get_size() { return pos_; } int set_prev_big_segment_scn(const share::SCN prev_scn); - int acquire_segment_log_buf(const char *&submit_buf, - int64_t &submit_buf_len, - const ObTxLogBlockHeader &block_header, - const ObTxLogType big_segment_log_type, - ObTxBigSegmentBuf *big_segment_buf = nullptr); + int acquire_segment_log_buf(const ObTxLogType big_segment_log_type, ObTxBigSegmentBuf *big_segment_buf = nullptr); private: - int serialize_log_block_header_(const int64_t replay_hint, - const ObTxLogBlockHeader &block_header, - const logservice::ObReplayBarrierType barrier_type = - logservice::ObReplayBarrierType::NO_NEED_BARRIER); - int deserialize_log_block_header_(int64_t &replay_hint, ObTxLogBlockHeader &block_header); + int serialize_log_block_header_(); + int deserialize_log_block_header_(); int update_next_log_pos_(); // skip log body if cur_log_type_ is UNKNOWN (depend on // DESERIALIZE_HEADER in ob_unify_serialize.h) DISALLOW_COPY_AND_ASSIGN(ObTxLogBlock); - + bool inited_; + logservice::ObLogBaseHeader log_base_header_; + ObTxLogBlockHeader header_; ObTxAdaptiveLogBuf fill_buf_; const char *replay_buf_; int64_t len_; int64_t pos_; ObTxLogType cur_log_type_; - logservice::ObReplayBarrierType cur_block_barrier_type_; ObTxCbArgArray cb_arg_array_; ObTxBigSegmentBuf *big_segment_buf_; diff --git a/src/storage/tx/ob_tx_log_adapter.cpp b/src/storage/tx/ob_tx_log_adapter.cpp index d9bcb9982a..fc9462cfd8 100644 --- a/src/storage/tx/ob_tx_log_adapter.cpp +++ b/src/storage/tx/ob_tx_log_adapter.cpp @@ -185,6 +185,21 @@ int ObLSTxLogAdapter::get_max_decided_scn(SCN &scn) return ret; } +int ObLSTxLogAdapter::get_palf_committed_max_scn(share::SCN &scn) const +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(log_handler_) || !log_handler_->is_valid()) { + ret = OB_INVALID_ARGUMENT; + TRANS_LOG(WARN, "invalid argument", K(ret), KP(log_handler_)); + } else if (OB_FAIL(log_handler_->get_max_decided_scn_as_leader(scn))) { + TRANS_LOG(WARN, "get palf committed_max_scn fail", K(ret)); + } else if (!scn.is_valid_and_not_min()) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "get an invalid scn", K(ret), K(scn)); + } + return ret; +} + int ObLSTxLogAdapter::get_append_mode_initial_scn(share::SCN &ref_scn) { int ret = OB_SUCCESS; diff --git a/src/storage/tx/ob_tx_log_adapter.h b/src/storage/tx/ob_tx_log_adapter.h index 5619df06f0..a2cfce4f29 100644 --- a/src/storage/tx/ob_tx_log_adapter.h +++ b/src/storage/tx/ob_tx_log_adapter.h @@ -67,7 +67,7 @@ public: virtual int get_role(bool &is_leader, int64_t &epoch) = 0; virtual int get_max_decided_scn(share::SCN &scn) = 0; - + virtual int get_palf_committed_max_scn(share::SCN &scn) const = 0; virtual int get_append_mode_initial_scn(share::SCN &ref_scn) = 0; /** @@ -111,6 +111,7 @@ public: const bool need_nonblock); int get_role(bool &is_leader, int64_t &epoch); int get_max_decided_scn(share::SCN &scn); + int get_palf_committed_max_scn(share::SCN &scn) const; int get_append_mode_initial_scn(share::SCN &ref_scn); diff --git a/src/storage/tx/ob_tx_ls_log_writer.cpp b/src/storage/tx/ob_tx_ls_log_writer.cpp index d00f3f3d72..ebba071728 100644 --- a/src/storage/tx/ob_tx_ls_log_writer.cpp +++ b/src/storage/tx/ob_tx_ls_log_writer.cpp @@ -59,10 +59,10 @@ void ObTxLSLogLimit::decide_log_buf_size() LOG_BUF_SIZE = 0; ObLogBaseHeader base_header; ObTxLogHeader tx_header; - ObTxLogBlockHeader block_header(UINT64_MAX, INT64_MAX, INT64_MAX, common::ObAddr()); + ObTxLogBlockHeader block_header(UINT64_MAX, INT64_MAX, INT64_MAX, INT64_MAX, common::ObAddr()); ObTxStartWorkingLog sw_log(INT_MAX64); - // block_header.before_serialize(); + block_header.before_serialize(); // sw_log.before_serialize(); int64_t max_body_size = 0; LOG_BUF_SIZE = base_header.get_serialize_size() + tx_header.get_serialize_size() + block_header.get_serialize_size(); diff --git a/src/storage/tx/ob_tx_ls_log_writer.h b/src/storage/tx/ob_tx_ls_log_writer.h index 6cedba6e19..bd486f5d64 100644 --- a/src/storage/tx/ob_tx_ls_log_writer.h +++ b/src/storage/tx/ob_tx_ls_log_writer.h @@ -115,15 +115,13 @@ int ObTxLSLogCb::serialize_ls_log(T &ls_log, } else { logservice::ObLogBaseHeader base_header(logservice::ObLogBaseType::TRANS_SERVICE_LOG_BASE_TYPE, barrier_type, replay_hint); - ObTxLogBlockHeader block_header; + ObTransID fake_tx_id(0); // fake a invalid tx_id + ObTxLogBlockHeader block_header(1, 1, 1, fake_tx_id, ObAddr()); ObTxLogHeader tx_header(T::LOG_TYPE); - // if (OB_FAIL(block_header.before_serialize())) { - // TRANS_LOG(WARN, "[TxLsLogWriter] before serialize block header error", KR(ret), - // K(block_header)); - // } else if (OB_FAIL(ls_log.before_serialize())) { - // TRANS_LOG(WARN, "[TxLsLogWriter] before serialize block header error", KR(ret), K(ls_log)); - // } else - if (OB_FAIL(base_header.serialize(log_buf_, ObTxLSLogLimit::LOG_BUF_SIZE, pos_))) { + if (OB_FAIL(block_header.before_serialize())) { + TRANS_LOG(WARN, "[TxLsLogWriter] before serialize block header error", KR(ret), + K(block_header)); + } else if (OB_FAIL(base_header.serialize(log_buf_, ObTxLSLogLimit::LOG_BUF_SIZE, pos_))) { TRANS_LOG(WARN, "[TxLsLogWriter] serialize base header error", KR(ret), KP(log_buf_), K(pos_)); } else if (OB_FAIL(block_header.serialize(log_buf_, ObTxLSLogLimit::LOG_BUF_SIZE, pos_))) { diff --git a/src/storage/tx/ob_tx_msg.cpp b/src/storage/tx/ob_tx_msg.cpp index fac87aa32a..f208e1bbf0 100644 --- a/src/storage/tx/ob_tx_msg.cpp +++ b/src/storage/tx/ob_tx_msg.cpp @@ -133,7 +133,7 @@ OB_DEF_SERIALIZE_SIZE(ObTxRollbackSPMsg) { int len = 0; len += ObTxMsg::get_serialize_size(); - LST_DO_CODE(OB_UNIS_ADD_LEN, savepoint_, op_sn_, branch_id_); + LST_DO_CODE(OB_UNIS_ADD_LEN, savepoint_, op_sn_, tx_seq_base_); if (OB_NOT_NULL(tx_ptr_)) { OB_UNIS_ADD_LEN(true); OB_UNIS_ADD_LEN(*tx_ptr_); @@ -149,7 +149,7 @@ OB_DEF_SERIALIZE(ObTxRollbackSPMsg) { int ret = ObTxMsg::serialize(buf, buf_len, pos); if (OB_SUCC(ret)) { - LST_DO_CODE(OB_UNIS_ENCODE, savepoint_, op_sn_, branch_id_); + LST_DO_CODE(OB_UNIS_ENCODE, savepoint_, op_sn_, tx_seq_base_); if (OB_NOT_NULL(tx_ptr_)) { OB_UNIS_ENCODE(true); OB_UNIS_ENCODE(*tx_ptr_); @@ -166,7 +166,7 @@ OB_DEF_DESERIALIZE(ObTxRollbackSPMsg) { int ret = ObTxMsg::deserialize(buf, data_len, pos); if (OB_SUCC(ret)) { - LST_DO_CODE(OB_UNIS_DECODE, savepoint_, op_sn_, branch_id_); + LST_DO_CODE(OB_UNIS_DECODE, savepoint_, op_sn_, tx_seq_base_); bool has_tx_ptr = false; OB_UNIS_DECODE(has_tx_ptr); if (has_tx_ptr) { diff --git a/src/storage/tx/ob_tx_msg.h b/src/storage/tx/ob_tx_msg.h index 333649471d..b2f62f084f 100644 --- a/src/storage/tx/ob_tx_msg.h +++ b/src/storage/tx/ob_tx_msg.h @@ -246,8 +246,7 @@ namespace transaction ObTxMsg(ROLLBACK_SAVEPOINT), savepoint_(), op_sn_(-1), - //todo:后续branch_id使用方式确定后,需要相应修改 - branch_id_(-1), + tx_seq_base_(0), tx_ptr_(NULL), flag_(USE_ASYNC_RESP), specified_from_scn_() @@ -262,8 +261,7 @@ namespace transaction } ObTxSEQ savepoint_; int64_t op_sn_; - //todo:后期设计中操作编号是否等于branch_id - int64_t branch_id_; + int64_t tx_seq_base_; const ObTxDesc *tx_ptr_; uint8_t flag_; ObTxSEQ specified_from_scn_; @@ -274,8 +272,8 @@ namespace transaction const static uint8_t ROLLBACK_FOR_TRANSFER = 0x02; bool is_valid() const; INHERIT_TO_STRING_KV("txMsg", ObTxMsg, - K_(savepoint), K_(op_sn), K_(branch_id), K_(flag), K_(specified_from_scn), - KP_(tx_ptr)); + K_(savepoint), K_(op_sn), K_(tx_seq_base), K_(flag), + K_(specified_from_scn), KP_(tx_ptr)); OB_UNIS_VERSION(1); }; diff --git a/src/storage/tx/ob_tx_redo_submitter.h b/src/storage/tx/ob_tx_redo_submitter.h new file mode 100644 index 0000000000..a6507386e2 --- /dev/null +++ b/src/storage/tx/ob_tx_redo_submitter.h @@ -0,0 +1,402 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_TRANSACTION_OB_TRANS_REDO_SUBMITTER +#define OCEANBASE_TRANSACTION_OB_TRANS_REDO_SUBMITTER + +#include "ob_trans_part_ctx.h" + +namespace oceanbase +{ +namespace transaction +{ +class ObPartTransCtx; + +class ObTxRedoSubmitter +{ +public: + ObTxRedoSubmitter(ObPartTransCtx &tx_ctx, + memtable::ObMemtableCtx &mt_ctx) : + tx_ctx_(tx_ctx), + mt_ctx_(mt_ctx), + tx_id_(tx_ctx.get_trans_id()), + ls_id_(tx_ctx.get_ls_id()), + log_block_(NULL), + log_cb_(NULL), + helper_(NULL), + from_all_list_(false), + flush_all_(false), + write_seq_no_(), + serial_final_(false), + submit_if_not_full_(true), + submit_cb_list_idx_(-1), + submit_out_cnt_(0), + submitted_scn_() + {} + ~ObTxRedoSubmitter(); + // submit, will traversal all callback-list + int submit(const bool flush_all, const bool is_final, const bool display_blocked_info = false); + // fill log_block, and if full submit out and continue to fill + int fill(ObTxLogBlock &log_block, memtable::ObRedoLogSubmitHelper &helper, const bool display_blocked_info = true); + // parallel submit, only traversal writter's callback-list + int parallel_submit(const ObTxSEQ &write_seq); + int get_submitted_cnt() const { return submit_out_cnt_; } + share::SCN get_submitted_scn() const { return submitted_scn_; } +private: + // common submit redo pipeline + // prepare -> fill -> submit_out -> after_submit + // ^ | + // | | + // \----------<--next-round-----------/ + // the loop will breaks in cases of: + // 1. no data need to be logged, all TxNode is logged + // 2. reach a blocking TxNode whose memtable is blocked to log + // 3. big row, the log_block can not hold the row + // 4. exception: memory allocated failure + int _submit_redo_pipeline_(const bool display_blocked_info); + int prepare_(); + int fill_log_block_(memtable::ObTxFillRedoCtx &ctx); + int submit_log_block_out_(const int64_t replay_hint, bool &submitted); + int after_submit_redo_out_(); +public: + TO_STRING_KV(K_(tx_id), K_(ls_id), K_(from_all_list), K_(flush_all), + K_(write_seq_no), K_(serial_final), + K_(submit_if_not_full), K_(submit_out_cnt), K_(submit_cb_list_idx)); +private: + ObPartTransCtx &tx_ctx_; + memtable::ObMemtableCtx &mt_ctx_; + const ObTransID tx_id_; + const share::ObLSID ls_id_; + ObTxLogBlock *log_block_; + ObTxLogCb *log_cb_; + memtable::ObRedoLogSubmitHelper *helper_; + // for writer thread submit, only submit single list + // for freeze or switch leader or commit, submit from all list + bool from_all_list_; + // whether flush all logs before can return + bool flush_all_; + // writer seq_no, used to select logging callback-list + ObTxSEQ write_seq_no_; + // whether is submitting the final serial log + bool serial_final_; + // wheter submit out if log_block is not full filled + bool submit_if_not_full_; + // submit from which list, use by wirte thread logging + int submit_cb_list_idx_; + // the count of logs this submitter submitted out + int submit_out_cnt_; + // last submitted log scn + share::SCN submitted_scn_; +}; + +#define FLUSH_REDO_TRACE_LEVEL DEBUG +#define FLUSH_REDO_TRACE(fmt, ...) TRANS_LOG(FLUSH_REDO_TRACE_LEVEL, "[REDO FLUSH]" fmt, K(ret), KPC(this), ## __VA_ARGS__); + +ObTxRedoSubmitter::~ObTxRedoSubmitter() +{ + if (log_cb_) { + tx_ctx_.return_redo_log_cb(log_cb_); + log_cb_ = NULL; + } +} + +// use by: +// flush redo after mvcc_write when txn has switched to parallel_logging +// +// the caller hold TxCtx's FlushRedo read Lock +int ObTxRedoSubmitter::parallel_submit(const ObTxSEQ &write_seq_no) +{ + int ret = OB_SUCCESS; + from_all_list_ = false; + flush_all_ = false; + write_seq_no_ = write_seq_no; + ObTxLogBlock log_block; + log_block_ = &log_block; + memtable::ObRedoLogSubmitHelper helper; + helper_ = &helper; + submit_if_not_full_ = true; + bool do_submit = false; + memtable::ObCallbackListLogGuard log_lock_guard; + if (OB_FAIL(mt_ctx_.get_log_guard(write_seq_no, log_lock_guard, submit_cb_list_idx_))) { + if (OB_NEED_RETRY == ret) { + // give up, lock conflict + ret = OB_SUCCESS; + } else if (OB_EAGAIN == ret) { + // others need flush firstly + // TODO: try flush others out and retry + if (TC_REACH_TIME_INTERVAL(5_s)) { + TRANS_LOG(WARN, "blocked by other list has smaller wirte epoch unlogged", + K(write_seq_no), K(tx_ctx_.get_trans_id())); + } + ret = OB_SUCCESS; + } else if (OB_ENTRY_NOT_EXIST == ret) { + // no callback to log + ret = OB_SUCCESS; + } else { + TRANS_LOG(WARN, "get log lock guard fail", K(ret)); + } + } else { + do_submit = true; + } + if (do_submit) { + ret = _submit_redo_pipeline_(false); + } + return ret; +} + +// used by: +// - flush redo after mvcc_write when Txn has not switched to parallel_logging +// - freeze submit redo +// - when parallel logging, at switch to follower or commit time flush all redos +// +// the caller has hold TransCtx's FlushRedo write Lock +// which ensure no writer thread is logging +// and also hold TransCtx's CtxLock, which is safe to operate in the flush pipline +int ObTxRedoSubmitter::submit(const bool flush_all, const bool is_final, const bool display_blocked_info) +{ + int ret = OB_SUCCESS; + from_all_list_ = true; + flush_all_ = flush_all; + serial_final_ = is_final; + ObTxLogBlock log_block; + log_block_ = &log_block; + memtable::ObRedoLogSubmitHelper helper; + helper_ = &helper; + submit_if_not_full_ = true; + ret = _submit_redo_pipeline_(display_blocked_info); + FLUSH_REDO_TRACE("serial submit done"); + return ret; +} + +// used by: +// - switch to follower when parallel logging is not enabled for this txn +// - commit time flush redos when parallel logging is not enabled for this txn +// +// the caller has hold TxCtx's CtxLock +// the caller has hold TxCtx's FlushRedo write Lock +int ObTxRedoSubmitter::fill(ObTxLogBlock &block, + memtable::ObRedoLogSubmitHelper &helper, + const bool display_blocked_info) +{ + int ret = OB_SUCCESS; + from_all_list_ = true; + flush_all_ = true; + log_block_ = █ + helper_ = &helper; + submit_if_not_full_ = false; + ret = _submit_redo_pipeline_(display_blocked_info); + FLUSH_REDO_TRACE("serial submit and fill done"); + return ret; +} + +int ObTxRedoSubmitter::_submit_redo_pipeline_(const bool display_blocked_info) +{ + int ret = OB_SUCCESS; + memtable::ObTxFillRedoCtx ctx; + ctx.tx_id_ = tx_id_; + const bool parallel_logging = tx_ctx_.is_parallel_logging(); + ctx.write_seq_no_ = write_seq_no_; + const bool is_parallel_logging = tx_ctx_.is_parallel_logging(); + bool stop = false; + int fill_ret = OB_SUCCESS; + while (OB_SUCC(ret) && !stop) { + if (submit_if_not_full_ && OB_FAIL(prepare_())) { + if (OB_TX_NOLOGCB != ret) { + TRANS_LOG(WARN, "prepare for submit log fail", K(ret)); + } + } else { + bool skip_submit = false; + fill_ret = fill_log_block_(ctx); + if (ctx.fill_count_ <= 0) { + // no data to flush, or no data can be flushed + ret = fill_ret; + stop = true; + skip_submit = true; + } else if (OB_SUCCESS == fill_ret) { + // this means all redo filled, no remians + ret = fill_ret; + stop = true; + // this is just fill log block, if all is filled + // won't submit out + skip_submit = !submit_if_not_full_; + } else if (OB_BUF_NOT_ENOUGH == fill_ret) { + if (ctx.fill_count_ == 0) { + // BIG_ROW has been handled in `fill_log_block_` + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "should not reach here", K(ret)); + ob_abort(); + } else { + fill_ret = OB_SUCCESS; + } + } else if (OB_BLOCK_FROZEN == fill_ret) { + if (is_parallel_logging) { + // should flush and continue retry other list + } else { + // serial logging, retry will failed with BLOCK_FROZEN also + stop = true; + ret = fill_ret; + } + } else if (OB_ITER_END == fill_ret) { + // blocked by other list, current list remains + if (is_parallel_logging) { + // should flush and retry others + } else { + // serial logging, shouldn't reach here + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "oops! fatal panic", K(ret), KPC(this)); + usleep(1000); + ob_abort(); + } + } else if (OB_EAGAIN == fill_ret) { + // this list is all filled, but others remains + if (is_parallel_logging) { + // should flush and retry others + } else { + // serial logging, shouldn't reach here + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "oops! fatal panic", K(ret), KPC(this)); + usleep(1000); + ob_abort(); + } + } else { + stop = true; + ret = fill_ret; + } + // start submit out filled log block + bool submitted = false; + int submit_ret = OB_SUCCESS; + if (ctx.fill_count_ > 0 && !skip_submit) { + const int64_t replay_hint = ctx.tx_id_.get_id() + (parallel_logging ? ctx.list_idx_ : 0); + submit_ret = submit_log_block_out_(replay_hint, submitted); + if (OB_SUCCESS == submit_ret) { + submit_ret = after_submit_redo_out_(); + } + } + if (stop) { + ret = submit_ret == OB_SUCCESS ? ret : submit_ret; + } else { + if (serial_final_ && submitted) { + stop = true; + } else if (!flush_all_) { + stop = true; + } + if (stop) { + // return fill_ret if submit success + ret = (submit_ret == OB_SUCCESS) ? fill_ret : submit_ret; + } else { + // if submit failed, should stop + ret = submit_ret; + } + } + } + } + if (OB_UNLIKELY(display_blocked_info) && fill_ret == OB_BLOCK_FROZEN) { + if (TC_REACH_TIME_INTERVAL(5_s)) { + TRANS_LOG(INFO, "[display-blocked-info]", "submit_redo_ctx", ctx); + } + } + FLUSH_REDO_TRACE("submit pipeline done", K(ctx), K_(tx_ctx)); + return ret; +} + +int ObTxRedoSubmitter::submit_log_block_out_(const int64_t replay_hint, bool &submitted) +{ + int ret = OB_SUCCESS; + submitted = false; + if (OB_NOT_NULL(log_cb_) && OB_FAIL(log_cb_->reserve_callbacks(helper_->callbacks_.count()))) { + TRANS_LOG(WARN, "log cb reserve callbacks space fail", K(ret)); + } else { + share::SCN submitted_scn; + helper_->callback_redo_submitted_ = false; + bool has_hold_ctx_lock = from_all_list_; + ret = tx_ctx_.submit_redo_log_out(*log_block_, log_cb_, *helper_, replay_hint, has_hold_ctx_lock, submitted_scn); + OB_ASSERT(log_cb_ == NULL); + if (submitted_scn.is_valid()) { + submitted = true; + ++submit_out_cnt_; + submitted_scn_ = submitted_scn; + } + } + FLUSH_REDO_TRACE("after submit redo log out", K(replay_hint), K(submitted)); + return ret; +} + +int ObTxRedoSubmitter::after_submit_redo_out_() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(mt_ctx_.log_submitted(*helper_))) { + TRANS_LOG(WARN, "callback memctx fail", K(ret)); + usleep(1000); + ob_abort(); + } + helper_->reset(); + return ret; +} + +// allocate/reserve resource for `after_submit_log_out_` +int ObTxRedoSubmitter::prepare_() +{ + int ret = OB_SUCCESS; + if (OB_NOT_NULL(log_cb_)) { + ret = OB_ERR_UNEXPECTED; + ob_abort(); + } else { + ret = tx_ctx_.prepare_for_submit_redo(log_cb_, *log_block_, serial_final_); + } + return ret; +} + +int ObTxRedoSubmitter::fill_log_block_(memtable::ObTxFillRedoCtx &ctx) +{ + int ret = OB_SUCCESS; + bool need_retry = false; + do { + need_retry = false; + ObTxRedoLog log(tx_ctx_.get_cluster_version()); + ret = log_block_->prepare_mutator_buf(log); + ctx.buf_ = log.get_mutator_buf(); + ctx.buf_len_ = log.get_mutator_size(); + ctx.buf_pos_ = 0; + ctx.helper_ = helper_; + ctx.skip_lock_node_ = false; + ctx.all_list_ = from_all_list_; + ctx.fill_count_ = 0; + ctx.list_idx_ = submit_cb_list_idx_; + int64_t start_ts = ObTimeUtility::fast_current_time(); + ret = mt_ctx_.fill_redo_log(ctx); + ctx.fill_time_ = ObTimeUtility::fast_current_time() - start_ts; + int save_ret = ret; + int64_t real_buf_pos = ctx.fill_count_ > 0 ? ctx.buf_pos_ : 0; + if (OB_FAIL(log_block_->finish_mutator_buf(log, real_buf_pos))) { + } else if (OB_UNLIKELY(real_buf_pos == 0 && (OB_BUF_NOT_ENOUGH == save_ret))) { + // extend log_block for big row + if (OB_FAIL(log_block_->extend_log_buf())) { + TRANS_LOG(WARN, "extend log buffer failed", K(ret), K(ctx), KPC_(log_block), KPC(this)); + if (OB_ALLOCATE_MEMORY_FAILED != ret) { + ret = OB_ERR_TOO_BIG_ROWSIZE; + } + } else { + need_retry = true; + TRANS_LOG(INFO, "extend log buffer success", K(ctx.buf_pos_), KPC(this)); + } + } else if (real_buf_pos > 0 && OB_FAIL(log_block_->add_new_log(log))) { + } else { + ret = save_ret; + } + } while(need_retry); + FLUSH_REDO_TRACE("after fill redo", K(ctx)); + return ret; +} +} // transaction +} // oceanbase +#undef FLUSH_REDO_LOG_TRACE +#endif // OCEANBASE_TRANSACTION_OB_TRANS_REDO_SUBMITTER diff --git a/src/storage/tx/ob_tx_replay_executor.cpp b/src/storage/tx/ob_tx_replay_executor.cpp index 909bdaf79f..4222df901d 100644 --- a/src/storage/tx/ob_tx_replay_executor.cpp +++ b/src/storage/tx/ob_tx_replay_executor.cpp @@ -40,24 +40,25 @@ int ObTxReplayExecutor::execute(storage::ObLS *ls, const int skip_pos, const palf::LSN &lsn, const SCN &log_timestamp, - const int64_t &replay_hint, - const ObLSID &ls_id, - const int64_t &tenant_id) + const logservice::ObLogBaseHeader &base_header, + const ObLSID &ls_id) { int ret = OB_SUCCESS; - ObTxReplayExecutor replay_executor(ls, ls_tx_srv, lsn, log_timestamp); + const uint64_t tenant_id = ls->get_tenant_id(); + ObTxReplayExecutor replay_executor(ls, + ls_id, + tenant_id, + ls_tx_srv, + lsn, + log_timestamp, + base_header); if (OB_ISNULL(ls) || OB_ISNULL(ls_tx_srv) || OB_ISNULL(buf) || size <= 0 || !log_timestamp.is_valid() || !lsn.is_valid()) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(ERROR, "invaild arguments", K(replay_executor), K(buf), K(size)); - } else if (OB_FAIL(replay_executor.do_replay_(buf, - size, - skip_pos, - replay_hint, - ls_id, - tenant_id))) { - TRANS_LOG(WARN, "replay_executor.do_replay failed", - K(replay_executor), K(buf), K(size), K(skip_pos), K(replay_hint), K(ls_id), K(tenant_id)); + } else if (OB_FAIL(replay_executor.do_replay_(buf, size, skip_pos))) { + TRANS_LOG(WARN, "replay_executor.do_replay failed", K(ret), + K(replay_executor), K(buf), K(size), K(skip_pos), K(ls_id), K(tenant_id)); } else { if (log_timestamp <= ls->get_ls_wrs_handler()->get_ls_weak_read_ts()) { SCN min_log_service_scn; @@ -69,7 +70,6 @@ int ObTxReplayExecutor::execute(storage::ObLS *ls, K(buf), K(size), K(skip_pos), - K(replay_hint), K(min_log_service_scn), K(ls_id), K(tenant_id)); @@ -78,12 +78,7 @@ int ObTxReplayExecutor::execute(storage::ObLS *ls, return ret; } -int ObTxReplayExecutor::do_replay_(const char *buf, - const int64_t size, - const int skip_pos, - const int64_t &replay_hint, - const share::ObLSID &ls_id, - const int64_t &tenant_id) +int ObTxReplayExecutor::do_replay_(const char *buf, const int64_t size, const int skip_pos) { int ret = OB_SUCCESS; if (OB_ISNULL(buf) || size <= 0) { @@ -91,17 +86,14 @@ int ObTxReplayExecutor::do_replay_(const char *buf, TRANS_LOG(ERROR, "invaild arguments", KPC(this), K(buf), K(size)); } else if (OB_SUCC(prepare_replay_(buf, size, skip_pos))) { ObTxLogHeader header; + ObTxLogType log_type; first_created_ctx_ = false; - while (OB_SUCC(ret)) { - if (OB_FAIL(try_get_tx_ctx_( - log_block_header_.get_tx_id(), - tenant_id, - ls_id))) { - TRANS_LOG(WARN, "try get tx ctx failed", K(ret), K(replay_hint), K(log_block_header_)); + if (OB_FAIL(try_get_tx_ctx_())) { + TRANS_LOG(WARN, "try get tx ctx failed", K(ret), KPC(this)); } else if (OB_ISNULL(ctx_)) { + // StartWorkingLog if (OB_FAIL(log_block_.get_next_log(header))) { - if (OB_ITER_END == ret) { ret = OB_SUCCESS; break; @@ -109,114 +101,33 @@ int ObTxReplayExecutor::do_replay_(const char *buf, TRANS_LOG(WARN, "[Replay Tx] get_next_log error in replay_buf", KPC(this)); } } - } else { - if (OB_FAIL(ctx_->iter_next_log_for_replay(log_block_, header, log_ts_ns_))) { - if (OB_ITER_END == ret) { - ret = OB_SUCCESS; - break; - } else { - TRANS_LOG(WARN, "[Replay Tx] get_next_log error in replay_buf", KPC(this)); - } + } else if (OB_FAIL(iter_next_log_for_replay_(header))) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + break; } + TRANS_LOG(WARN, "[Replay Tx] get_next_log error in replay_buf", KPC(this)); } - - if(OB_FAIL(ret)) { - //do nothing - } else if (ctx_ != nullptr - && OB_FAIL(ctx_->validate_replay_log_entry_no(first_created_ctx_, - log_block_header_.get_log_entry_no(), - log_ts_ns_))) { - TRANS_LOG(WARN, "validate log entry no failed", K(ret), K(header), K(first_created_ctx_), - K(log_block_header_), K(log_ts_ns_), K(ls_id), K(replay_hint)); - } else { - tx_part_log_no_ += 1; // mark the replaying log_no in same log_ts - - switch (header.get_tx_log_type()) { - case ObTxLogType::TX_REDO_LOG: { + if (OB_SUCC(ret)) { + log_type = header.get_tx_log_type(); + if (log_type == ObTxLogType::TX_REDO_LOG) { if (OB_FAIL(before_replay_redo_())) { TRANS_LOG(WARN, "[Replay Tx] start replay redo log failed", K(ret)); } else if (!can_replay()) { ret = OB_STATE_NOT_MATCH; - TRANS_LOG(ERROR, "can not replay tx log", K(ret), K(header), K(replay_hint), K(tenant_id), K(ls_id)); + TRANS_LOG(ERROR, "can not replay tx log", K(ret), K(header), KPC(this)); } else if (OB_FAIL(replay_redo_())) { TRANS_LOG(WARN, "[Replay Tx] replay redo log error", K(ret)); } - break; - } - case ObTxLogType::TX_ROLLBACK_TO_LOG: { - if (OB_FAIL(replay_rollback_to_())) { - TRANS_LOG(WARN, "[Replay Tx] replay rollbackTo log error", KR(ret)); - } - break; - } - case ObTxLogType::TX_ACTIVE_INFO_LOG: { - if (OB_FAIL(replay_active_info_())) { - TRANS_LOG(WARN, "replay active_state error", K(ret)); - } - break; - } - case ObTxLogType::TX_COMMIT_INFO_LOG: { - if (OB_FAIL(replay_commit_info_())) { - TRANS_LOG(WARN, "[Replay Tx] replay commit info log error", K(ret)); - } - break; - } - case ObTxLogType::TX_PREPARE_LOG: { - if (OB_FAIL(replay_prepare_())) { - TRANS_LOG(WARN, "[Replay Tx] replay prepare log error", K(ret)); - } - break; - } - case ObTxLogType::TX_COMMIT_LOG: { - if (OB_FAIL(replay_commit_())) { - TRANS_LOG(WARN, "[Replay Tx] replay commit log error", K(ret)); - } - break; - } - case ObTxLogType::TX_ABORT_LOG: { - if (OB_FAIL(replay_abort_())) { - TRANS_LOG(WARN, "[Replay Tx] replay abort log error", K(ret)); - } - break; - } - case ObTxLogType::TX_CLEAR_LOG: { - if (OB_FAIL(replay_clear_())) { - TRANS_LOG(WARN, "[Replay Tx] replay clear log error", K(ret)); - } - break; - } - case ObTxLogType::TX_START_WORKING_LOG: { + } else if (log_type == ObTxLogType::TX_START_WORKING_LOG) { if (OB_FAIL(replay_start_working_())) { TRANS_LOG(WARN, "[Replay Tx] replay clear log error", KR(ret)); } - break; - } - case ObTxLogType::TX_MULTI_DATA_SOURCE_LOG: { - if (OB_FAIL(replay_multi_source_data_())) { - TRANS_LOG(WARN, "[Replay Tx] replay multi source data log error", KR(ret)); - } - break; - } - case ObTxLogType::TX_RECORD_LOG: { - if (OB_FAIL(replay_record_())) { - TRANS_LOG(WARN, "[Replay Tx] replay record log error", KR(ret)); - } - break; - } - case ObTxLogType::TX_BIG_SEGMENT_LOG: { - if (OB_FAIL(ctx_->replay_one_part_of_big_segment(lsn_, log_ts_ns_, tx_part_log_no_))) { - TRANS_LOG(WARN, "[Replay Tx] replay big segment log error", KR(ret)); - } - break; - } - default: { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "[Replay Tx] Unknown Log Type in replay buf", - K(header.get_tx_log_type()), KPC(this)); - } + } else { + ret = replay_tx_log_(log_type); } } - TRANS_LOG(DEBUG, "[Replay Tx] Replay One Tx Log", K(header.get_tx_log_type()), K(ret)); + TRANS_LOG(DEBUG, "[Replay Tx] Replay One Tx Log", K(log_type), K(ret), K_(log_ts_ns)); } finish_replay_(ret); rewrite_replay_retry_code_(ret); @@ -242,40 +153,135 @@ OB_NOINLINE int ObTxReplayExecutor::errsim_tx_replay_() return ret; } +int ObTxReplayExecutor::iter_next_log_for_replay_(ObTxLogHeader &header) +{ + int ret = OB_SUCCESS; + if (log_block_.get_header().get_tx_id().get_id() == base_header_.get_replay_hint()) { + ret = ctx_->iter_next_log_for_replay(log_block_, header, log_ts_ns_); + } else { + bool contain_bigsegment = false; + ret = log_block_.get_next_log(header, NULL, &contain_bigsegment); + if (contain_bigsegment) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "parallel replay should not contains bigsegment!", KR(ret)); + } + } + return ret; +} + +int ObTxReplayExecutor::replay_tx_log_(const ObTxLogType log_type) +{ + int ret = OB_SUCCESS; + tx_part_log_no_ += 1; // mark the replaying log_no in same log_ts + switch (log_type) { + case ObTxLogType::TX_ROLLBACK_TO_LOG: { + if (OB_FAIL(replay_rollback_to_())) { + TRANS_LOG(WARN, "[Replay Tx] replay rollbackTo log error", KR(ret)); + } + break; + } + case ObTxLogType::TX_ACTIVE_INFO_LOG: { + if (OB_FAIL(replay_active_info_())) { + TRANS_LOG(WARN, "replay active_state error", K(ret)); + } + break; + } + case ObTxLogType::TX_COMMIT_INFO_LOG: { + if (OB_FAIL(replay_commit_info_())) { + TRANS_LOG(WARN, "[Replay Tx] replay commit info log error", K(ret)); + } + break; + } + case ObTxLogType::TX_PREPARE_LOG: { + if (OB_FAIL(replay_prepare_())) { + TRANS_LOG(WARN, "[Replay Tx] replay prepare log error", K(ret)); + } + break; + } + case ObTxLogType::TX_COMMIT_LOG: { + if (OB_FAIL(replay_commit_())) { + TRANS_LOG(WARN, "[Replay Tx] replay commit log error", K(ret)); + } + break; + } + case ObTxLogType::TX_ABORT_LOG: { + if (OB_FAIL(replay_abort_())) { + TRANS_LOG(WARN, "[Replay Tx] replay abort log error", K(ret)); + } + break; + } + case ObTxLogType::TX_CLEAR_LOG: { + if (OB_FAIL(replay_clear_())) { + TRANS_LOG(WARN, "[Replay Tx] replay clear log error", K(ret)); + } + break; + } + case ObTxLogType::TX_MULTI_DATA_SOURCE_LOG: { + if (OB_FAIL(replay_multi_source_data_())) { + TRANS_LOG(WARN, "[Replay Tx] replay multi source data log error", KR(ret)); + } + break; + } + case ObTxLogType::TX_RECORD_LOG: { + if (OB_FAIL(replay_record_())) { + TRANS_LOG(WARN, "[Replay Tx] replay record log error", KR(ret)); + } + break; + } + case ObTxLogType::TX_BIG_SEGMENT_LOG: { + if (OB_FAIL(ctx_->replay_one_part_of_big_segment(lsn_, log_ts_ns_, tx_part_log_no_))) { + TRANS_LOG(WARN, "[Replay Tx] replay big segment log error", KR(ret)); + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "[Replay Tx] Unknown Log Type in replay buf", + K(log_type), KPC(this)); + usleep(100000); + ob_abort(); + } + } + return ret; +} + int ObTxReplayExecutor::prepare_replay_(const char *buf, const int64_t &size, const int skip_pos) { int ret = OB_SUCCESS; if (OB_FAIL(errsim_tx_replay_())) { TRANS_LOG(WARN, "errsim for tx replay", K(ret), K(log_ts_ns_), K(lsn_)); - } else if (OB_FAIL(log_block_.init(buf, size, skip_pos, log_block_header_))) { - TRANS_LOG(ERROR, "TxLogBlock init error", K(log_block_), K(log_block_header_)); + } else if (OB_FAIL(log_block_.init_for_replay(buf, size, skip_pos))) { + TRANS_LOG(ERROR, "TxLogBlock init error", K(log_block_)); + } else { + replay_queue_ = base_header_.get_replay_hint() - log_block_.get_header().get_tx_id().get_id(); + replaying_log_entry_no_ = log_block_.get_header().get_log_entry_no(); } return ret; } -int ObTxReplayExecutor::try_get_tx_ctx_(int64_t tx_id, int64_t tenant_id, const ObLSID &ls_id) +int ObTxReplayExecutor::try_get_tx_ctx_() { int ret = OB_SUCCESS; - - ObTransID trans_id(tx_id); + ObTransID tx_id = log_block_.get_header().get_tx_id(); // replay ls log without part_ctx if (ctx_ != nullptr) { first_created_ctx_ = false; - } else if (trans_id.is_valid() && nullptr == ctx_) { - + } else if (tx_id.is_valid() && nullptr == ctx_) { if (OB_FAIL(ls_tx_srv_->get_tx_ctx(tx_id, true, ctx_)) && OB_TRANS_CTX_NOT_EXIST != ret) { TRANS_LOG(WARN, "[Replay Tx] get tx ctx from ctx_mgr failed", K(ret), K(tx_id), KP(ctx_)); } else if (OB_TRANS_CTX_NOT_EXIST == ret) { ret = OB_SUCCESS; bool tx_ctx_existed = false; - common::ObAddr scheduler = log_block_header_.get_scheduler(); + common::ObAddr scheduler = log_block_.get_header().get_scheduler(); + // since 4.3, cluster version in log_block_header + const uint64_t cluster_version = log_block_.get_header().get_cluster_version(); ObTxCreateArg arg(true, /* for_replay */ false, /* for_special_tx */ - tenant_id, + tenant_id_, tx_id, - ls_id, - log_block_header_.get_org_cluster_id(), - GET_MIN_CLUSTER_VERSION(), + ls_id_, + log_block_.get_header().get_org_cluster_id(), + cluster_version, 0, /*session_id*/ scheduler, INT64_MAX, /*trans_expired_time_*/ @@ -283,15 +289,21 @@ int ObTxReplayExecutor::try_get_tx_ctx_(int64_t tx_id, int64_t tenant_id, const if (OB_FAIL(ls_tx_srv_->create_tx_ctx(arg, tx_ctx_existed, ctx_))) { TRANS_LOG(WARN, "get_tx_ctx error", K(ret), K(tx_id), KP(ctx_)); } else { - first_created_ctx_ = true; + first_created_ctx_ = !tx_ctx_existed; } } - - if (OB_SUCC(ret)) { - ctx_->push_repalying_log_ts(log_ts_ns_); + if (OB_NOT_NULL(ctx_)) { + if (replaying_log_entry_no_ == 0) { + ret = ctx_->set_replay_completeness(true); + } else if (is_tx_log_replay_queue() && ctx_->is_replay_completeness_unknown()) { + TRANS_LOG(INFO, "tx replay from middle", K(tx_id), K_(replaying_log_entry_no), K_(lsn), K_(log_ts_ns)); + ret = ctx_->set_replay_completeness(false); + } + if (OB_SUCC(ret) && is_tx_log_replay_queue()) { + ret = ctx_->push_repalying_log_ts(log_ts_ns_, replaying_log_entry_no_ == 0); + } } } - return ret; } @@ -299,9 +311,10 @@ int ObTxReplayExecutor::before_replay_redo_() { int ret = OB_SUCCESS; if (!has_redo_) { + const bool parallel_replay = !is_tx_log_replay_queue(); if (OB_ISNULL(ctx_) || OB_ISNULL(mt_ctx_ = ctx_->get_memtable_ctx())) { ret = OB_INVALID_ARGUMENT; - } else if (mt_ctx_->replay_begin(log_ts_ns_)) { + } else if (mt_ctx_->replay_begin(parallel_replay, log_ts_ns_)) { TRANS_LOG(ERROR, "[Replay Tx] replay_begin fail or mt_ctx_ is NULL", K(ret), K(mt_ctx_)); } else { has_redo_ = true; @@ -313,20 +326,26 @@ int ObTxReplayExecutor::before_replay_redo_() void ObTxReplayExecutor::finish_replay_(const int retcode) { if (has_redo_) { + const int16_t callback_list_idx = replay_queue_; if (OB_SUCCESS != retcode) { mt_ctx_->replay_end(false, /*is_replay_succ*/ + callback_list_idx, log_ts_ns_); - TRANS_LOG_RET(WARN, OB_EAGAIN, "[Replay Tx]Tx Redo replay error, rollback to start", K(*this)); + TRANS_LOG_RET(WARN, OB_EAGAIN, "[Replay Tx]Tx Redo replay error, rollback to start", + K(callback_list_idx), KPC(this)); } else { mt_ctx_->replay_end(true, /*is_replay_succ*/ + callback_list_idx, log_ts_ns_); // TRANS_LOG(INFO, "[Replay Tx] Tx Redo replay success, commit sub_trans", K(*this)); } } if (nullptr != ctx_) { - if (OB_SUCCESS == retcode) { - ctx_->push_replayed_log_ts(log_ts_ns_, lsn_); + if (is_tx_log_replay_queue()) { + if (OB_SUCCESS == retcode) { + ctx_->push_replayed_log_ts(log_ts_ns_, lsn_, replaying_log_entry_no_); + } } if (OB_SUCCESS != retcode) { ctx_->print_trace_log(); @@ -346,7 +365,9 @@ int ObTxReplayExecutor::replay_redo_() int tmp_ret = OB_SUCCESS; ObTxRedoLogTempRef temp_ref; ObTxRedoLog redo_log(temp_ref); - + if (is_tx_log_replay_queue()) { + tx_part_log_no_ += 1; // redo is compound with tx log, mark part_log_no is required + } if (OB_ISNULL(ls_)) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "[Replay Tx] ls should not be null", K(ret), K(ls_)); @@ -355,7 +376,12 @@ int ObTxReplayExecutor::replay_redo_() K(log_ts_ns_)); } else if (OB_FAIL(replay_redo_in_memtable_(redo_log))) { TRANS_LOG(WARN, "[Replay Tx] replay redo in memtable error", K(ret), K(lsn_), K(log_ts_ns_)); - } else if (OB_FAIL(ctx_->replay_redo_in_ctx(redo_log, lsn_, log_ts_ns_, tx_part_log_no_))) { + } else if (OB_FAIL(ctx_->replay_redo_in_ctx(redo_log, + lsn_, + log_ts_ns_, + tx_part_log_no_, + is_tx_log_replay_queue(), + log_block_.get_header().is_serial_final()))) { TRANS_LOG(WARN, "[Replay Tx] replay redo in tx_ctx error", K(ret), K(lsn_), K(log_ts_ns_)); // } else if (first_created_ctx_ && redo_log.get_log_no() > 0) { // // replay a commited tx in recovery process @@ -366,10 +392,9 @@ int ObTxReplayExecutor::replay_redo_() // K(first_created_ctx_), K(redo_log.get_log_no())); // ctx_->supplement_undo_actions_if_exist(); } - - if (OB_SUCC(ret) && OB_TMP_FAIL(mt_ctx_->remove_callbacks_for_fast_commit())) { + if (OB_SUCC(ret) && OB_TMP_FAIL(mt_ctx_->remove_callbacks_for_fast_commit(replay_queue_, share::SCN::minus(log_ts_ns_, 1)))) { TRANS_LOG(WARN, "[Replay Tx] remove callbacks for fast commit", K(ret), K(tmp_ret), - K(lsn_), K(log_ts_ns_), K(*mt_ctx_)); + K(replay_queue_), K(lsn_), K(log_ts_ns_), K(*mt_ctx_)); } return ret; @@ -378,13 +403,15 @@ int ObTxReplayExecutor::replay_redo_() int ObTxReplayExecutor::replay_rollback_to_() { int ret = OB_SUCCESS; + const bool tx_queue = is_tx_log_replay_queue(); ObTxRollbackToLog log; + const bool pre_barrier = base_header_.need_pre_replay_barrier(); if (OB_FAIL(log_block_.deserialize_log_body(log))) { TRANS_LOG(WARN, "[Replay Tx] deserialize log body error", KR(ret), "log_type", "RollbackTo", K(lsn_), K(log_ts_ns_)); - } else if (OB_FAIL(ctx_->replay_rollback_to(log, lsn_, log_ts_ns_, tx_part_log_no_))) { + } else if (OB_FAIL(ctx_->replay_rollback_to(log, lsn_, log_ts_ns_, tx_part_log_no_, tx_queue, pre_barrier))) { TRANS_LOG(WARN, "[Replay Tx] replay rollback_to in tx_ctx error", KR(ret), K(lsn_), - K(log_ts_ns_)); + K(log_ts_ns_), K(tx_queue)); } return ret; } @@ -411,10 +438,10 @@ int ObTxReplayExecutor::replay_start_working_() ObTxStartWorkingLogTempRef temp_ref; ObTxStartWorkingLog start_working_log(temp_ref); if (OB_FAIL(log_block_.deserialize_log_body(start_working_log))) { - TRANS_LOG(WARN, "[Replay Tx] deserialize log body error", KR(ret), K(start_working_log), + TRANS_LOG(WARN, "[Replay Tx] deserialize start_working log body error", KR(ret), K(start_working_log), K(lsn_), K(log_ts_ns_)); } else if (OB_FAIL(ls_tx_srv_->replay_start_working_log(start_working_log, log_ts_ns_))) { - TRANS_LOG(WARN, "[Replay Tx] replay active_info in tx_ctx error", KR(ret), K(lsn_), + TRANS_LOG(WARN, "[Replay Tx] replay start_working log in tx_ctx error", KR(ret), K(lsn_), K(log_ts_ns_)); } return ret; @@ -452,12 +479,11 @@ int ObTxReplayExecutor::replay_commit_info_() int ret = OB_SUCCESS; ObTxCommitInfoLogTempRef temp_ref; ObTxCommitInfoLog commit_info_log(temp_ref); - + const bool pre_barrier = base_header_.need_pre_replay_barrier(); if (OB_FAIL(log_block_.deserialize_log_body(commit_info_log))) { TRANS_LOG(WARN, "[Replay Tx] deserialize log body error", K(ret), K(commit_info_log), K(lsn_), K(log_ts_ns_)); - } else if (OB_FAIL( - ctx_->replay_commit_info(commit_info_log, lsn_, log_ts_ns_, tx_part_log_no_))) { + } else if (OB_FAIL(ctx_->replay_commit_info(commit_info_log, lsn_, log_ts_ns_, tx_part_log_no_, pre_barrier))) { TRANS_LOG(WARN, "[Replay Tx] replay commit_info in tx_ctx error", K(ret), K(lsn_), K(log_ts_ns_)); } @@ -495,7 +521,7 @@ int ObTxReplayExecutor::replay_commit_() log_ts_ns_, tx_part_log_no_, replay_compact_version))) { - TRANS_LOG(WARN, "[Replay Tx] replay commit in tx_ctx error", K(ret), K(lsn_), K(log_ts_ns_)); + TRANS_LOG(WARN, "[Replay Tx] replay commit in tx_ctx error", K(ret), KPC(this)); } return ret; @@ -620,6 +646,8 @@ int ObTxReplayExecutor::replay_redo_in_memtable_(ObTxRedoLog &redo) return ret; } +#define TX_REPLAY_LOG(log_level, fmt, ...) \ + TRANS_LOG(log_level, "[Replay Tx]" fmt, K(ret), ## __VA_ARGS__, KPC(this)); int ObTxReplayExecutor::replay_one_row_in_memtable_(ObMutatorRowHeader &row_head, memtable::ObMemtableMutatorIterator *mmi_ptr) { @@ -627,40 +655,33 @@ int ObTxReplayExecutor::replay_one_row_in_memtable_(ObMutatorRowHeader &row_head lib::Worker::CompatMode mode; ObTabletHandle tablet_handle; const bool is_update_mds_table = false; - if (OB_FAIL(ls_->replay_get_tablet(row_head.tablet_id_, log_ts_ns_, is_update_mds_table, tablet_handle))) { if (OB_OBSOLETE_CLOG_NEED_SKIP == ret) { - ctx_->force_no_need_replay_checksum(); + ctx_->force_no_need_replay_checksum(!is_tx_log_replay_queue(), log_ts_ns_); ret = OB_SUCCESS; - TRANS_LOG(WARN, "[Replay Tx] tablet gc, skip this log entry", K(ret), K(row_head.tablet_id_), - KP(ls_), K(log_ts_ns_), K(tx_part_log_no_), K(ctx_)); + TX_REPLAY_LOG(WARN, "tablet gc, skip this log entry", K(row_head.tablet_id_)); } else if (OB_EAGAIN == ret) { - TRANS_LOG(INFO, "[Replay Tx] tablet not ready, retry this log entry", K(ret), K(row_head.tablet_id_), - KP(ls_), K(log_ts_ns_), K(tx_part_log_no_), K(ctx_)); + TX_REPLAY_LOG(INFO, "tablet not ready, retry this log entry", K(row_head.tablet_id_)); } else { - TRANS_LOG(INFO, "[Replay Tx] get tablet failed, retry this log entry", K(ret), K(row_head.tablet_id_), - KP(ls_), K(log_ts_ns_), K(tx_part_log_no_), K(ctx_)); + TX_REPLAY_LOG(INFO, "get tablet failed, retry this log entry", K(row_head.tablet_id_)); ret = OB_EAGAIN; } } else if (OB_FAIL(logservice::ObTabletReplayExecutor::replay_check_restore_status(tablet_handle, false/*update_tx_data*/))) { if (OB_NO_NEED_UPDATE == ret) { - ctx_->check_no_need_replay_checksum(log_ts_ns_); + ctx_->check_no_need_replay_checksum(log_ts_ns_, replay_queue_); ret = OB_SUCCESS; if (REACH_TIME_INTERVAL(1000 * 1000)) { - TRANS_LOG(INFO, "[Replay Tx] Not need replay, skip this log entry", K(row_head.tablet_id_), - K(log_ts_ns_), K(tx_part_log_no_)); + TX_REPLAY_LOG(INFO, "Not need replay, skip this log entry", K(row_head.tablet_id_)); } } else if (OB_EAGAIN == ret) { if (REACH_TIME_INTERVAL(1000 * 1000)) { - TRANS_LOG(INFO, "[Replay Tx] tablet not ready, retry this log entry", K(ret), K(row_head.tablet_id_), - K(log_ts_ns_), K(tx_part_log_no_)); + TX_REPLAY_LOG(INFO, "tablet not ready, retry this log entry", K(row_head.tablet_id_)); } } else { - TRANS_LOG(WARN, "[Replay Tx] replay check restore status error", K(ret), K(row_head.tablet_id_), - K(log_ts_ns_), K(tx_part_log_no_)); + TX_REPLAY_LOG(WARN, "replay check restore status error", K(row_head.tablet_id_)); } } else if (OB_FAIL(get_compat_mode_(row_head.tablet_id_, mode))) { - TRANS_LOG(WARN, "[Replay Tx] get compat mode error", K(ret), K(mode)); + TX_REPLAY_LOG(WARN, "get compat mode error", K(mode)); } else { ObTablet *tablet = tablet_handle.get_obj(); storage::ObStoreCtx storeCtx; @@ -670,7 +691,6 @@ int ObTxReplayExecutor::replay_one_row_in_memtable_(ObMutatorRowHeader &row_head *mt_ctx_, ctx_->get_trans_id() ); - storeCtx.replay_log_scn_ = log_ts_ns_; storeCtx.tablet_id_ = row_head.tablet_id_; storeCtx.ls_ = ls_; @@ -683,7 +703,7 @@ int ObTxReplayExecutor::replay_one_row_in_memtable_(ObMutatorRowHeader &row_head TRANS_LOG(WARN, "[Replay Tx] replay row failed.", K(ret), K(mt_ctx_), K(row_head.tablet_id_)); } else if (OB_NO_NEED_UPDATE == ret) { - ctx_->check_no_need_replay_checksum(log_ts_ns_); + ctx_->check_no_need_replay_checksum(log_ts_ns_, replay_queue_); ret = OB_SUCCESS; TRANS_LOG(DEBUG, "[Replay Tx] Not need replay row becase of no_need_update", K(log_ts_ns_), K(tx_part_log_no_), K(row_head.tablet_id_)); @@ -755,7 +775,7 @@ int ObTxReplayExecutor::replay_row_(storage::ObStoreCtx &store_ctx, TRANS_LOG(WARN, "[Replay Tx] this is not a ObMemtable", K(ret), KP(mem_ptr), KPC(mem_ptr), KP(mmi_ptr)); } else if (FALSE_IT(timeguard.click("get_memtable"))) { - } else if (OB_FAIL(data_mem_ptr->replay_row(store_ctx, mmi_ptr))) { + } else if (OB_FAIL(data_mem_ptr->replay_row(store_ctx, log_ts_ns_, mmi_ptr))) { TRANS_LOG(WARN, "[Replay Tx] replay row error", K(ret)); } else if (OB_FAIL(data_mem_ptr->set_max_end_scn(log_ts_ns_))) { // for freeze log_ts , may be TRANS_LOG(WARN, "[Replay Tx] set memtable max end log ts failed", K(ret), KP(data_mem_ptr)); @@ -769,7 +789,7 @@ int ObTxReplayExecutor::replay_row_(storage::ObStoreCtx &store_ctx, // in a freeze memtable which has a smaller end ts than this log. // // The rollback operation must hold write_ref to make memtable stay in memory. - mt_ctx_->rollback_redo_callbacks(log_ts_ns_); + mt_ctx_->rollback_redo_callbacks(replay_queue_, log_ts_ns_); } return ret; } @@ -790,7 +810,7 @@ int ObTxReplayExecutor::replay_lock_(storage::ObStoreCtx &store_ctx, ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "[Replay Tx] get lock memtable failed", K(ret), K(handle)); } else if (FALSE_IT(timeguard.click("get_memtable"))) { - } else if (OB_FAIL(memtable->replay_row(store_ctx, mmi_ptr))) { + } else if (OB_FAIL(memtable->replay_row(store_ctx, log_ts_ns_, mmi_ptr))) { TRANS_LOG(WARN, "[Replay Tx] replay lock row error", K(ret)); } else { TRANS_LOG(DEBUG, "[Replay Tx] replay row in lock memtable success", KP(memtable)); diff --git a/src/storage/tx/ob_tx_replay_executor.h b/src/storage/tx/ob_tx_replay_executor.h index 833163c1dd..f65e822a70 100644 --- a/src/storage/tx/ob_tx_replay_executor.h +++ b/src/storage/tx/ob_tx_replay_executor.h @@ -57,27 +57,48 @@ public: const int skip_pos, const palf::LSN &lsn, const share::SCN &log_timestamp, - const int64_t &replay_hint, - const share::ObLSID &ls_id, - const int64_t &tenant_id); + const logservice::ObLogBaseHeader &base_header, + const share::ObLSID &ls_id); public: TO_STRING_KV(KP(ctx_), KP(ls_), + K(ls_id_), + K(tenant_id_), KP(ls_tx_srv_), KP(mt_ctx_), K(log_block_), + K_(replay_queue), + K_(replaying_log_entry_no), K(lsn_), - K(log_ts_ns_)); + K(log_ts_ns_), + K(base_header_)); private: ObTxReplayExecutor(storage::ObLS *ls, + const share::ObLSID &ls_id, + const uint64_t tenant_id, storage::ObLSTxService *ls_tx_srv, const palf::LSN &lsn, - const share::SCN &log_timestamp) - : ctx_(nullptr), ls_(ls), ls_tx_srv_(ls_tx_srv), lsn_(lsn), - log_ts_ns_(log_timestamp), mmi_ptr_(nullptr), mt_ctx_(nullptr), first_created_ctx_(false), - has_redo_(false), tx_part_log_no_(0), mvcc_row_count_(0), table_lock_row_count_(0) + const share::SCN &log_timestamp, + const logservice::ObLogBaseHeader &base_header) + : ctx_(nullptr), + ls_(ls), + ls_id_(ls_id), + tenant_id_(tenant_id), + ls_tx_srv_(ls_tx_srv), + replay_queue_(0), + replaying_log_entry_no_(0), + lsn_(lsn), + log_ts_ns_(log_timestamp), + mmi_ptr_(nullptr), + mt_ctx_(nullptr), + first_created_ctx_(false), + has_redo_(false), + tx_part_log_no_(0), + mvcc_row_count_(0), + table_lock_row_count_(0), + base_header_(base_header) {} ~ObTxReplayExecutor() { ob_free(mmi_ptr_); } @@ -85,18 +106,17 @@ private: private: int do_replay_(const char *buf, const int64_t size, - const int skip_pos, - const int64_t &replay_hint, - const share::ObLSID &ls_id, - const int64_t &tenant_id); + const int skip_pos); int prepare_replay_(const char *buf, const int64_t &size, const int skip_pos); - int try_get_tx_ctx_(int64_t tx_id, int64_t tenant_id, const share::ObLSID &ls_id); + int try_get_tx_ctx_(); + int iter_next_log_for_replay_(ObTxLogHeader &header); int before_replay_redo_(); void finish_replay_(const int retcode); int errsim_tx_replay_(); int replay_redo_(); + int replay_tx_log_(const ObTxLogType log_type); int replay_rollback_to_(); int replay_active_info_(); int replay_commit_info_(); @@ -107,7 +127,7 @@ private: int replay_start_working_(); int replay_multi_source_data_(); int replay_record_(); - + bool is_tx_log_replay_queue() const { return replay_queue_ == 0; } int replay_redo_in_memtable_(ObTxRedoLog &redo); virtual int replay_one_row_in_memtable_(memtable::ObMutatorRowHeader& row_head, memtable::ObMemtableMutatorIterator *mmi_ptr); @@ -129,10 +149,13 @@ private: ReplayTxCtx *ctx_; storage::ObLS *ls_; + share::ObLSID ls_id_; + uint64_t tenant_id_; storage::ObLSTxService *ls_tx_srv_; - ObTxLogBlockHeader log_block_header_; ObTxLogBlock log_block_; + int16_t replay_queue_; + int64_t replaying_log_entry_no_; palf::LSN lsn_; share::SCN log_ts_ns_; @@ -146,6 +169,7 @@ private: // memtable::ObMemtable * mem_store_; int64_t mvcc_row_count_; int64_t table_lock_row_count_; + const logservice::ObLogBaseHeader &base_header_; }; } } // namespace oceanbase diff --git a/src/storage/tx/ob_tx_retain_ctx_mgr.cpp b/src/storage/tx/ob_tx_retain_ctx_mgr.cpp index a4c05ae2ef..be257976a5 100644 --- a/src/storage/tx/ob_tx_retain_ctx_mgr.cpp +++ b/src/storage/tx/ob_tx_retain_ctx_mgr.cpp @@ -119,19 +119,17 @@ int ObIRetainCtxCheckFunctor::del_retain_ctx() int ObMDSRetainCtxFunctor::init(ObPartTransCtx *ctx, RetainCause cause, - const SCN &final_log_ts, - palf::LSN final_log_lsn) + const SCN &final_log_ts) { int ret = OB_SUCCESS; - if (!final_log_ts.is_valid() || !final_log_lsn.is_valid()) { + if (!final_log_ts.is_valid()) { ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid argument", K(ret), K(final_log_ts), K(final_log_lsn)); + TRANS_LOG(WARN, "invalid argument", K(ret), K(final_log_ts)); } else if (OB_FAIL(ObIRetainCtxCheckFunctor::init(ctx, cause))) { TRANS_LOG(WARN, "init retain ctx check functor failed", K(ret)); } else { final_log_ts_ = final_log_ts; - final_log_lsn_ = final_log_lsn; } return ret; @@ -162,8 +160,7 @@ int ObMDSRetainCtxFunctor::operator()(ObLS *ls, ObTxRetainCtxMgr *retain_mgr) bool ObMDSRetainCtxFunctor::is_valid() { - return ObIRetainCtxCheckFunctor::is_valid() && final_log_ts_.is_valid() - && final_log_lsn_.is_valid(); + return ObIRetainCtxCheckFunctor::is_valid() && final_log_ts_.is_valid(); } void ObTxRetainCtxMgr::reset() diff --git a/src/storage/tx/ob_tx_retain_ctx_mgr.h b/src/storage/tx/ob_tx_retain_ctx_mgr.h index e0e2648594..23a2b6bd27 100644 --- a/src/storage/tx/ob_tx_retain_ctx_mgr.h +++ b/src/storage/tx/ob_tx_retain_ctx_mgr.h @@ -16,7 +16,6 @@ #include "lib/list/ob_dlink_node.h" #include "lib/list/ob_list.h" #include "lib/lock/ob_spin_rwlock.h" -#include "logservice/palf/lsn.h" #include "storage/tx/ob_trans_define.h" namespace oceanbase @@ -80,19 +79,16 @@ public: ObMDSRetainCtxFunctor() : ObIRetainCtxCheckFunctor() { final_log_ts_.reset(); - final_log_lsn_.reset(); } int init(ObPartTransCtx *ctx, RetainCause cause, - const share::SCN &final_log_ts, - palf::LSN final_log_lsn); + const share::SCN &final_log_ts); virtual int operator()(storage::ObLS *ls, ObTxRetainCtxMgr *retain_mgr) override; virtual bool is_valid() override; private: share::SCN final_log_ts_; - palf::LSN final_log_lsn_; }; typedef common::ObDList RetainCtxList; diff --git a/src/storage/tx/ob_tx_stat.cpp b/src/storage/tx/ob_tx_stat.cpp index 912b2ce1de..98eb774df8 100644 --- a/src/storage/tx/ob_tx_stat.cpp +++ b/src/storage/tx/ob_tx_stat.cpp @@ -47,6 +47,10 @@ void ObTxStat::reset() xid_.reset(); coord_.reset(); last_request_ts_ = OB_INVALID_TIMESTAMP; + busy_cbs_cnt_ = 0; + replay_completeness_ = -1; + serial_final_scn_.reset(); + callback_list_stats_.reset(); } int ObTxStat::init(const common::ObAddr &addr, const ObTransID &tx_id, const uint64_t tenant_id, const bool has_decided, @@ -61,7 +65,10 @@ int ObTxStat::init(const common::ObAddr &addr, const ObTransID &tx_id, const int64_t session_id, const common::ObAddr &scheduler, const bool is_exiting, const ObXATransID &xid, const share::ObLSID &coord, const int64_t last_request_ts, - SCN start_scn, SCN end_scn, SCN rec_scn, bool transfer_blocking) + SCN start_scn, SCN end_scn, SCN rec_scn, bool transfer_blocking, + const int busy_cbs_cnt, + int replay_completeness, + share::SCN serial_final_scn) { int ret = OB_SUCCESS; if (is_inited_) { @@ -102,6 +109,9 @@ int ObTxStat::init(const common::ObAddr &addr, const ObTransID &tx_id, end_scn_ = end_scn; rec_scn_ = rec_scn; transfer_blocking_ = transfer_blocking; + busy_cbs_cnt_ = busy_cbs_cnt; + replay_completeness_ = replay_completeness; + serial_final_scn_ = serial_final_scn; } return ret; } diff --git a/src/storage/tx/ob_tx_stat.h b/src/storage/tx/ob_tx_stat.h index 9781bdc64d..dc25c954b9 100644 --- a/src/storage/tx/ob_tx_stat.h +++ b/src/storage/tx/ob_tx_stat.h @@ -15,9 +15,37 @@ #include "ob_trans_define.h" #include "common/ob_range.h" +#include "lib/utility/ob_print_utils.h" namespace oceanbase { +namespace memtable +{ +struct ObTxCallbackListStat +{ + int id_; + share::SCN sync_scn_; + share::SCN checksum_scn_; + int length_; + int logged_; + int removed_; + void reset() {} + DECLARE_TO_STRING + { + int64_t pos = 0; + BUF_PRINTF("(%d,%d,%d,%d,%ld,%ld)", + id_, + length_, + logged_, + removed_, + (sync_scn_.is_valid() ? sync_scn_.get_val_for_inner_table_field() : -1), + (checksum_scn_.is_valid() ? checksum_scn_.get_val_for_inner_table_field() : -1)); + return pos; + } +}; + +} // memtable namespace + namespace transaction { @@ -39,16 +67,25 @@ struct ObTxStat const int64_t session_id, const common::ObAddr &scheduler, const bool is_exiting, const ObXATransID &xid, const share::ObLSID &coord, const int64_t last_request_ts, - share::SCN start_scn, share::SCN end_scn, share::SCN rec_scn, bool transfer_blocking); + share::SCN start_scn, share::SCN end_scn, share::SCN rec_scn, bool transfer_blocking, + const int busy_cbs_cnt, + int replay_completeness, + share::SCN serial_final_scn); TO_STRING_KV(K_(addr), K_(tx_id), K_(tenant_id), - K_(has_decided), K_(ls_id), K_(participants), - K_(tx_ctx_create_time), K_(tx_expired_time), K_(ref_cnt), - K_(last_op_sn), K_(pending_write), K_(state), K_(tx_type), - KP_(tx_ctx_addr), - K_(pending_log_size), K_(flushed_log_size), - K_(role_state), K_(session_id), - K_(scheduler_addr), K_(is_exiting), - K_(xid), K_(coord), K_(last_request_ts), K_(start_scn), K_(end_scn), K_(rec_scn), K_(transfer_blocking)); + K_(has_decided), K_(ls_id), K_(participants), + K_(tx_ctx_create_time), K_(tx_expired_time), K_(ref_cnt), + K_(last_op_sn), K_(pending_write), K_(state), K_(tx_type), + KP_(tx_ctx_addr), + K_(pending_log_size), K_(flushed_log_size), + K_(role_state), K_(session_id), + K_(scheduler_addr), K_(is_exiting), + K_(xid), K_(coord), K_(last_request_ts), + K_(xid), K_(coord), K_(last_request_ts), + K_(start_scn), K_(end_scn), K_(rec_scn), K_(transfer_blocking), + K_(busy_cbs_cnt), + K_(serial_final_scn), + K_(replay_completeness), + K_(callback_list_stats)); public: bool is_inited_; common::ObAddr addr_; @@ -79,6 +116,27 @@ public: share::SCN end_scn_; share::SCN rec_scn_; bool transfer_blocking_; + int busy_cbs_cnt_; + int replay_completeness_; + share::SCN serial_final_scn_; + ObSEArray callback_list_stats_; + struct CLStatsDisplay { + CLStatsDisplay(ObSEArray &stats): stats_(stats) {} + ObSEArray &stats_; + DECLARE_TO_STRING + { + int64_t pos = 0; + if (stats_.count() > 0) { + BUF_PRINTF("[id, length, logged, removed, sync_scn, checksum_scn]"); + BUF_PRINTO(stats_); + } + return pos; + } + }; + CLStatsDisplay get_callback_list_stats_displayer() + { + return CLStatsDisplay(callback_list_stats_); + } }; class ObTxLockStat diff --git a/src/storage/tx/ob_xa_service.cpp b/src/storage/tx/ob_xa_service.cpp index f5fb4db18e..c079234d36 100644 --- a/src/storage/tx/ob_xa_service.cpp +++ b/src/storage/tx/ob_xa_service.cpp @@ -2073,7 +2073,7 @@ int ObXAService::xa_rollback_all_changes(const ObXATransID &xid, ObTxDesc *&tx_d if (OB_FAIL(start_stmt(xid, 0/*unused session id*/, *tx_desc))) { TRANS_LOG(WARN, "xa start stmt fail", K(ret), K(xid), K(tx_id)); } else { - const transaction::ObTxSEQ savepoint = tx_desc->get_tx_seq(1); + const transaction::ObTxSEQ savepoint = tx_desc->get_min_tx_seq(); if (OB_FAIL(MTL(transaction::ObTransService *)->rollback_to_implicit_savepoint(*tx_desc, savepoint, stmt_expired_time, NULL))) { TRANS_LOG(WARN, "do savepoint rollback error", K(ret), K(xid), K(tx_id)); diff --git a/src/storage/tx_storage/ob_access_service.cpp b/src/storage/tx_storage/ob_access_service.cpp index ecaf327c7e..846d6ec6e2 100644 --- a/src/storage/tx_storage/ob_access_service.cpp +++ b/src/storage/tx_storage/ob_access_service.cpp @@ -125,6 +125,7 @@ int ObAccessService::pre_check_lock( param.expired_time_, /*timeout*/ tx_desc, snapshot, + 0,/*branch_id*/ write_flag, ctx_guard))) { LOG_WARN("fail to check query allowed", K(ret), K(ls_id)); @@ -162,6 +163,7 @@ int ObAccessService::lock_obj( param.expired_time_, /*timeout*/ tx_desc, snapshot, + 0, /*branch_id*/ write_flag, ctx_guard))) { LOG_WARN("fail to check query allowed", K(ret), K(ls_id)); @@ -200,6 +202,7 @@ int ObAccessService::unlock_obj( param.expired_time_, /*timeout*/ tx_desc, snapshot, + 0,/*branch_id*/ write_flag, ctx_guard))) { LOG_WARN("fail to check query allowed", K(ret), K(ls_id)); @@ -343,6 +346,7 @@ int ObAccessService::get_write_store_ctx_guard_( const int64_t timeout, transaction::ObTxDesc &tx_desc, const transaction::ObTxReadSnapshot &snapshot, + const int16_t branch_id, const concurrent_control::ObWriteFlag write_flag, ObStoreCtxGuard &ctx_guard, const transaction::ObTxSEQ &spec_seq_no) @@ -361,6 +365,7 @@ int ObAccessService::get_write_store_ctx_guard_( ObStoreCtx &ctx = ctx_guard.get_store_ctx(); ctx.ls_ = ls; ctx.timeout_ = timeout; + ctx.branch_ = branch_id; if (OB_FAIL(ls->get_write_store_ctx(tx_desc, snapshot, write_flag, ctx, spec_seq_no))) { LOG_WARN("can not get write store ctx", K(ret), K(ls_id), K(snapshot), K(tx_desc)); } @@ -575,6 +580,7 @@ int ObAccessService::check_write_allowed_( dml_param.timeout_, tx_desc, dml_param.snapshot_, + dml_param.branch_id_, dml_param.write_flag_, ctx_guard, dml_param.spec_seq_no_))) { diff --git a/src/storage/tx_storage/ob_access_service.h b/src/storage/tx_storage/ob_access_service.h index af60d3de7d..c4c3bce683 100644 --- a/src/storage/tx_storage/ob_access_service.h +++ b/src/storage/tx_storage/ob_access_service.h @@ -208,6 +208,7 @@ protected: const int64_t timeout, transaction::ObTxDesc &tx_desc, const transaction::ObTxReadSnapshot &snapshot, + const int16_t branch_id, const concurrent_control::ObWriteFlag write_flag, ObStoreCtxGuard &ctx_guard, const transaction::ObTxSEQ &spec_seq_no = transaction::ObTxSEQ::INVL()); diff --git a/src/storage/tx_storage/ob_checkpoint_service.cpp b/src/storage/tx_storage/ob_checkpoint_service.cpp index 5a9a6ec03c..2e6d765ce1 100644 --- a/src/storage/tx_storage/ob_checkpoint_service.cpp +++ b/src/storage/tx_storage/ob_checkpoint_service.cpp @@ -308,6 +308,7 @@ void ObCheckPointService::ObTraversalFlushTask::runTimerTask() ObLSIterator *iter = NULL; common::ObSharedGuard guard; ObLSService *ls_svr = MTL(ObLSService*); + ObCurTraceId::init(GCONF.self_addr_); if (OB_ISNULL(ls_svr)) { STORAGE_LOG(WARN, "mtl ObLSService should not be null", K(ret)); } else if (OB_FAIL(ls_svr->get_ls_iter(guard, ObLSGetMod::TXSTORAGE_MOD))) { @@ -340,6 +341,7 @@ void ObCheckPointService::ObTraversalFlushTask::runTimerTask() } } } + ObCurTraceId::reset(); } void ObCheckPointService::ObCheckClogDiskUsageTask::runTimerTask() diff --git a/src/storage/tx_table/ob_tx_ctx_table.cpp b/src/storage/tx_table/ob_tx_ctx_table.cpp index 8545e946ae..1d91bfd729 100644 --- a/src/storage/tx_table/ob_tx_ctx_table.cpp +++ b/src/storage/tx_table/ob_tx_ctx_table.cpp @@ -104,13 +104,15 @@ int ObTxCtxTableRecoverHelper::recover_one_tx_ctx_(transaction::ObLSTxCtxMgr* ls transaction::ObPartTransCtx *tx_ctx = NULL; bool tx_ctx_existed = true; common::ObAddr scheduler; + // since 4.3 cluster_version in ctx_info + uint64_t cluster_version = ctx_info.cluster_version_; transaction::ObTxCreateArg arg(true, /* for_replay */ false, MTL_ID(), ctx_info.tx_id_, ctx_info.ls_id_, ctx_info.cluster_id_, /* cluster_id */ - ctx_info.cluster_version_, + cluster_version, 0, /*session_id*/ scheduler, INT64_MAX, diff --git a/src/storage/tx_table/ob_tx_table_define.cpp b/src/storage/tx_table/ob_tx_table_define.cpp index d79dd489e4..ff1481e68a 100644 --- a/src/storage/tx_table/ob_tx_table_define.cpp +++ b/src/storage/tx_table/ob_tx_table_define.cpp @@ -113,8 +113,8 @@ int ObTxCtxTableInfo::serialize_(char *buf, TRANS_LOG(WARN, "serialize exec_info fail.", KR(ret), K(pos), K(buf_len)); } else if (OB_FAIL(table_lock_info_.serialize(buf, buf_len, pos))) { TRANS_LOG(WARN, "serialize exec_info fail.", KR(ret), K(pos), K(buf_len)); - } else if (OB_FAIL(serialization::encode(buf, buf_len, pos, cluster_version_))) { - TRANS_LOG(WARN, "encode cluster version failed", K(cluster_version_), K(buf_len), K(pos), K(ret)); + } else if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, cluster_version_))) { + TRANS_LOG(WARN, "encode cluster_version fail", K(buf_len), K(pos), K(ret)); } return ret; @@ -157,13 +157,11 @@ int ObTxCtxTableInfo::deserialize_(const char *buf, TRANS_LOG(WARN, "deserialize exec_info fail.", KR(ret), K(pos), K(buf_len)); } else if (OB_FAIL(table_lock_info_.deserialize(buf, buf_len, pos))) { TRANS_LOG(WARN, "deserialize exec_info fail.", KR(ret), K(pos), K(buf_len)); - } else if (pos >= buf_len) { - // for compatibility - if (OB_FAIL(GET_MIN_DATA_VERSION(MTL_ID(), cluster_version_))) { - TRANS_LOG(INFO, "get min data version failed", K(ret)); + } + if (OB_SUCC(ret) && buf_len > pos) { // has remains, continue to deserialize new members + if (OB_FAIL(serialization::decode_vi64(buf, buf_len, pos, &cluster_version_))) { + TRANS_LOG(WARN, "dencode cluster_version fail", K(buf_len), K(pos), K(ret)); } - } else if (OB_FAIL(serialization::decode(buf, buf_len, pos, cluster_version_))) { - TRANS_LOG(WARN, "encode cluster_version fail", K(cluster_version_), K(buf_len), K(pos), K(ret)); } return ret; @@ -187,10 +185,10 @@ int64_t ObTxCtxTableInfo::get_serialize_size_(void) const len += tx_id_.get_serialize_size(); len += ls_id_.get_serialize_size(); len += serialization::encoded_length_vi64(cluster_id_); + len += serialization::encoded_length_vi64(cluster_version_); len += (OB_NOT_NULL(tx_data_guard_.tx_data()) ? tx_data_guard_.tx_data()->get_serialize_size() : 0); len += exec_info_.get_serialize_size(); len += table_lock_info_.get_serialize_size(); - len += serialization::encoded_length(cluster_version_); return len; } diff --git a/src/storage/tx_table/ob_tx_table_define.h b/src/storage/tx_table/ob_tx_table_define.h index c9c3ee2613..e83dfd038f 100644 --- a/src/storage/tx_table/ob_tx_table_define.h +++ b/src/storage/tx_table/ob_tx_table_define.h @@ -91,21 +91,20 @@ public: tx_id_.reset(); ls_id_.reset(); cluster_id_ = OB_INVALID_CLUSTER_ID; + cluster_version_ = 0; tx_data_guard_.reset(); exec_info_.reset(); table_lock_info_.reset(); - cluster_version_ = 0; } void destroy() { reset(); } TO_STRING_KV(K_(tx_id), K_(ls_id), K_(cluster_id), K_(tx_data_guard), K_(exec_info), K_(cluster_version)); transaction::ObTransID tx_id_; share::ObLSID ls_id_; int64_t cluster_id_; + int64_t cluster_version_; ObTxDataGuard tx_data_guard_; transaction::ObTxExecInfo exec_info_; transaction::tablelock::ObTableLockInfo table_lock_info_; - // cluster version for compatibility - uint64_t cluster_version_; }; struct ObTxCtxTableMeta diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result index ac2002f5ed..db72c6d877 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result @@ -289,6 +289,7 @@ _enable_newsort _enable_new_sql_nio _enable_oracle_priv_check _enable_parallel_minor_merge +_enable_parallel_redo_logging _enable_parallel_table_creation _enable_partition_level_retry _enable_pkt_nio @@ -370,7 +371,9 @@ _optimizer_skip_scan_enabled _optimizer_sortmerge_join_enabled _parallel_max_active_sessions _parallel_min_message_pool +_parallel_redo_logging_trigger _parallel_server_sleep_time +_pdml_thread_cache_size _pipelined_table_function_memory_limit _print_sample_ppm _private_buffer_size diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_mysql.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_mysql.result index 7a4f18d1d1..a9126cea75 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_mysql.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_mysql.result @@ -582,7 +582,7 @@ end_scn bigint(20) unsigned NO NULL logging_blocked varchar(3) NO NULL freeze_clock bigint(20) NO NULL unsubmitted_count bigint(20) NO NULL -unsynced_count bigint(20) NO NULL +max_end_scn bigint(20) unsigned NO NULL write_ref_count bigint(20) NO NULL mem_used bigint(20) NO NULL hash_item_count bigint(20) NO NULL @@ -636,6 +636,10 @@ start_scn bigint(20) unsigned NO NULL end_scn bigint(20) unsigned NO NULL rec_scn bigint(20) unsigned NO NULL transfer_blocking tinyint(4) NO NULL +busy_cbs bigint(20) NO NULL +replay_complete bigint(20) NO NULL +serial_log_final_scn bigint(20) NO NULL +callback_list_stats longtext NO NULL select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_trans_stat; IF(count(*) >= 0, 1, 0) 1 diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_sys.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_sys.result index 76ece3f62d..af4ff4e43e 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_sys.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_virtual_table_in_sys.result @@ -631,7 +631,7 @@ end_scn bigint(20) unsigned NO NULL logging_blocked varchar(3) NO NULL freeze_clock bigint(20) NO NULL unsubmitted_count bigint(20) NO NULL -unsynced_count bigint(20) NO NULL +max_end_scn bigint(20) unsigned NO NULL write_ref_count bigint(20) NO NULL mem_used bigint(20) NO NULL hash_item_count bigint(20) NO NULL @@ -692,6 +692,10 @@ start_scn bigint(20) unsigned NO NULL end_scn bigint(20) unsigned NO NULL rec_scn bigint(20) unsigned NO NULL transfer_blocking tinyint(4) NO NULL +busy_cbs bigint(20) NO NULL +replay_complete bigint(20) NO NULL +serial_log_final_scn bigint(20) NO NULL +callback_list_stats longtext NO NULL select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_trans_stat; IF(count(*) >= 0, 1, 0) 1 diff --git a/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.cpp b/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.cpp index a45c93a5a1..131074658f 100644 --- a/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.cpp +++ b/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.cpp @@ -78,20 +78,25 @@ int ObAdminParserLogEntry::get_entry_header_(ObLogBaseHeader &header) LOG_WARN("deserialize ObLogBaseHeader failed", K(ret), K(pos_), K(buf_len_)); } else { str_arg_.log_stat_->log_base_header_size_ += (pos_ - tmp_pos); + if (str_arg_.flag_ == LogFormatFlag::NO_FORMAT + && str_arg_.flag_ != LogFormatFlag::STAT_FORMAT ) { + fprintf(stdout, ", BASE_HEADER:%s", to_cstring(header)); + } LOG_TRACE("get_entry_header success", K(header), K(pos_)); } return ret; } -int ObAdminParserLogEntry::parse_trans_service_log_(ObTxLogBlock &tx_log_block) +int ObAdminParserLogEntry::parse_trans_service_log_(ObTxLogBlock &tx_log_block, const logservice::ObLogBaseHeader &base_header) { int ret = OB_SUCCESS; str_arg_.log_stat_->total_tx_log_count_++; TxID tx_id; - ObTxLogBlockHeader tx_block_header; - if (OB_FAIL(tx_log_block.init_with_header(buf_, buf_len_, tx_id, tx_block_header))) { + ObTxLogBlockHeader &tx_block_header = tx_log_block.get_header(); + if (OB_FAIL(tx_log_block.init_for_replay(buf_, buf_len_))) { LOG_WARN("ObTxLogBlock init failed", K(ret)); + } else if (FALSE_IT(tx_id = tx_block_header.get_tx_id().get_id())) { } else if (str_arg_.filter_.is_tx_id_valid() && tx_id != str_arg_.filter_.get_tx_id()) { //just skip this LOG_TRACE("skip with tx_id", K(str_arg_), K(tx_id), K(block_name_), K(lsn_)); @@ -120,6 +125,20 @@ int ObAdminParserLogEntry::parse_trans_service_log_(ObTxLogBlock &tx_log_block) str_arg_.writer_ptr_->dump_key(block_name_); str_arg_.writer_ptr_->dump_key("LSN"); str_arg_.writer_ptr_->dump_int64((int64_t)(lsn_.val_)); + str_arg_.writer_ptr_->dump_key("ReplayHint"); + str_arg_.writer_ptr_->dump_int64(base_header.get_replay_hint()); + str_arg_.writer_ptr_->dump_key("ReplayBarrier"); + bool pre_b = base_header.need_pre_replay_barrier(); + bool post_b = base_header.need_post_replay_barrier(); + if (pre_b && post_b) { + str_arg_.writer_ptr_->dump_string("STRICT"); + } else if (pre_b) { + str_arg_.writer_ptr_->dump_string("PRE"); + } else if (post_b) { + str_arg_.writer_ptr_->dump_string("POST"); + } else { + str_arg_.writer_ptr_->dump_string("NONE"); + } } str_arg_.writer_ptr_->dump_key("TxID"); str_arg_.writer_ptr_->dump_int64(tx_id); @@ -610,13 +629,13 @@ int ObAdminParserLogEntry::parse_different_entry_type_(const logservice::ObLogBa if (oceanbase::logservice::ObLogBaseType::TRANS_SERVICE_LOG_BASE_TYPE == header.get_log_type()) { //TX_FORMAT only cares trans_log ObTxLogBlock log_block; - ret = parse_trans_service_log_(log_block); + ret = parse_trans_service_log_(log_block, header); } } else { switch (header.get_log_type()) { case oceanbase::logservice::ObLogBaseType::TRANS_SERVICE_LOG_BASE_TYPE: { ObTxLogBlock log_block; - ret = parse_trans_service_log_(log_block); + ret = parse_trans_service_log_(log_block, header); break; } case oceanbase::logservice::ObLogBaseType::STORAGE_SCHEMA_LOG_BASE_TYPE: { diff --git a/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.h b/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.h index 4c6011a1b9..355bfa4d7d 100644 --- a/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.h +++ b/tools/ob_admin/log_tool/parser/ob_admin_parser_log_entry.h @@ -48,7 +48,7 @@ public: private: int parse_different_entry_type_(const logservice::ObLogBaseHeader &header); int get_entry_header_(logservice::ObLogBaseHeader &header); - int parse_trans_service_log_(transaction::ObTxLogBlock &tx_log_block); + int parse_trans_service_log_(transaction::ObTxLogBlock &tx_log_block, const logservice::ObLogBaseHeader &base_header); int parse_schema_log_(); int parse_tablet_seq_sync_log_(); int parse_ddl_log_(); diff --git a/unittest/libobcdc/log_generator.h b/unittest/libobcdc/log_generator.h index 2d7cb67a40..8f018440ba 100644 --- a/unittest/libobcdc/log_generator.h +++ b/unittest/libobcdc/log_generator.h @@ -42,7 +42,7 @@ class ObTxLogBlockBuilder { public: ObTxLogBlockBuilder(const int64_t tx_id, const uint64_t cluster_id) - : tx_id_(tx_id), cluster_id_(cluster_id), log_entry_no_(0), tx_log_block_() {} + : tx_id_(tx_id), cluster_id_(cluster_id), cluster_version_(DATA_VERSION_4_3_0_0), log_entry_no_(0), tx_log_block_() {} ~ObTxLogBlockBuilder() {} public: int next_log_block(); @@ -55,7 +55,8 @@ public: int64_t get_log_entry_no() { return log_entry_no_; } private: TxID tx_id_; - uint64 cluster_id_; + uint64_t cluster_id_; + int64_t cluster_version_; int64_t log_entry_no_; ObTxLogBlock tx_log_block_; }; @@ -211,10 +212,10 @@ private: int ObTxLogBlockBuilder::next_log_block() { int ret = OB_SUCCESS; - ObTxLogBlockHeader block_header(cluster_id_, log_entry_no_, tx_id_, common::ObAddr()); tx_log_block_.reset(); - - if (OB_FAIL(tx_log_block_.init(tx_id_, block_header))) { + ObTxLogBlockHeader &block_header = tx_log_block_.get_header(); + block_header.init(cluster_id_, cluster_version_, log_entry_no_, tx_id_, common::ObAddr()); + if (OB_FAIL(tx_log_block_.init_for_fill())) { LOG_ERROR("init tx_log_block_ failed", KR(ret), K_(tx_id), K(block_header)); } else { log_entry_no_ ++; @@ -321,10 +322,11 @@ void ObTxLogGenerator::gen_commit_log() EXPECT_EQ(OB_SUCCESS, ls_info_arr.push_back(ls_info1)); EXPECT_EQ(OB_SUCCESS, ls_info_arr.push_back(ls_info2)); } - + ObArray checksum_signature; ObTxCommitLog commit_log( commit_version, checksum, + checksum_signature, inc_ls_arr, mds_arr, trans_type_, diff --git a/unittest/storage/memtable/mvcc/test_mvcc_callback.cpp b/unittest/storage/memtable/mvcc/test_mvcc_callback.cpp index 739b0572eb..a5a1bfc27a 100644 --- a/unittest/storage/memtable/mvcc/test_mvcc_callback.cpp +++ b/unittest/storage/memtable/mvcc/test_mvcc_callback.cpp @@ -14,7 +14,7 @@ #define private public #define protected public - +#include "storage/tx/ob_trans_part_ctx.h" #include "storage/memtable/ob_memtable.h" #include "storage/memtable/mvcc/ob_mvcc_trans_ctx.h" #include "storage/memtable/ob_memtable_context.h" @@ -33,10 +33,9 @@ class ObMockTxCallback : public ObITransCallback public: ObMockTxCallback(ObMemtable *mt, bool need_submit_log = true, - bool need_fill_redo = true, share::SCN scn = share::SCN::max_scn(), transaction::ObTxSEQ seq_no = transaction::ObTxSEQ::MAX_VAL()) - : ObITransCallback(need_fill_redo, need_submit_log), + : ObITransCallback(need_submit_log), mt_(mt), seq_no_(seq_no) { scn_ = scn; } virtual ObIMemtable* get_memtable() const override { return mt_; } @@ -44,7 +43,7 @@ public: virtual int checkpoint_callback() override; virtual int rollback_callback() override; virtual int calc_checksum(const share::SCN checksum_scn, - ObBatchChecksum *checksumer) override; + TxChecksum *checksumer) override; ObMemtable *mt_; transaction::ObTxSEQ seq_no_; @@ -99,12 +98,12 @@ class TestTxCallbackList : public ::testing::Test { public: TestTxCallbackList() - : seq_counter_(), + : seq_counter_(100, 0), mt_counter_(0), mt_ctx_(), cb_allocator_(), mgr_(mt_ctx_, cb_allocator_), - callback_list_(mgr_) {} + callback_list_(mgr_, 101) { } virtual void SetUp() override { mt_counter_ = 0; @@ -121,6 +120,12 @@ public: { mt_counter_ = 0; fast_commit_reserve_cnt_ = 0; + callback_list_.appended_ = 0; + callback_list_.removed_ = 0; + callback_list_.length_ = 0; + callback_list_.synced_ = 0; + callback_list_.logged_ = 0; + callback_list_.unlog_removed_ = 0; callback_list_.reset(); mgr_.reset(); TRANS_LOG(INFO, "teardown success"); @@ -137,13 +142,11 @@ public: ObMockTxCallback *create_callback(ObMemtable *mt, bool need_submit_log = true, - bool need_fill_redo = true, share::SCN scn = share::SCN::max_scn()) { auto seq_no = ++seq_counter_; ObMockTxCallback *cb = new ObMockTxCallback(mt, need_submit_log, - need_fill_redo, scn, seq_no); return cb; @@ -151,17 +154,14 @@ public: ObITransCallback *create_and_append_callback(ObMemtable *mt, bool need_submit_log = true, - bool need_fill_redo = true, share::SCN scn = share::SCN::max_scn()) { ObMockTxCallback *cb = create_callback(mt, need_submit_log, - need_fill_redo, scn); EXPECT_NE(NULL, (long)cb); EXPECT_EQ(OB_SUCCESS, callback_list_.append_callback(cb, false/*for_replay*/)); cb->need_submit_log_ = need_submit_log; - cb->need_fill_redo_ = need_fill_redo; return cb; } @@ -182,7 +182,7 @@ public: other.reset(); for (int64_t i = 1; i <= no; i++) { - other.add_bit(i); + other.add_bit(i + 100 /*seq's base is 100*/); } return res.equal(other); @@ -206,6 +206,8 @@ int64_t TestTxCallbackList::checkpoint_cnt_; int64_t TestTxCallbackList::rollback_cnt_; ObMockBitSet TestTxCallbackList::checksum_; +static bool has_remove = false; + int ObMockTxCallback::checkpoint_callback() { TestTxCallbackList::checkpoint_cnt_++; @@ -219,7 +221,7 @@ int ObMockTxCallback::rollback_callback() } int ObMockTxCallback::calc_checksum(const share::SCN checksum_scn, - ObBatchChecksum *) + TxChecksum *) { if (checksum_scn <= scn_) { TestTxCallbackList::checksum_.add_bit(seq_no_.get_seq()); @@ -238,17 +240,13 @@ TEST_F(TestTxCallbackList, remove_callback_on_failure) create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1); auto cb1 = create_and_append_callback(memtable, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); auto cb2 = create_and_append_callback(memtable, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); ObCallbackScope scope; int64_t removed_cnt = 0; @@ -256,7 +254,7 @@ TEST_F(TestTxCallbackList, remove_callback_on_failure) scope.end_ = ObITransCallbackIterator(cb2); EXPECT_EQ(false, scope.is_empty()); - EXPECT_EQ(OB_SUCCESS, callback_list_.sync_log_fail(scope, removed_cnt)); + EXPECT_EQ(OB_SUCCESS, callback_list_.sync_log_fail(scope, scn_1, removed_cnt)); EXPECT_EQ(2, removed_cnt); EXPECT_EQ(2, callback_list_.get_length()); @@ -270,15 +268,12 @@ TEST_F(TestTxCallbackList, remove_callback_by_tx_commit) create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); EXPECT_EQ(3, callback_list_.get_length()); @@ -296,15 +291,12 @@ TEST_F(TestTxCallbackList, remove_callback_by_tx_abort) scn_1.convert_for_logservice(1); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); EXPECT_EQ(3, callback_list_.get_length()); @@ -327,36 +319,27 @@ TEST_F(TestTxCallbackList, remove_callback_by_release_memtable) scn_100.convert_for_logservice(100); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable3, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable1, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_100 /*scn*/); EXPECT_EQ(9, callback_list_.get_length()); @@ -392,61 +375,53 @@ TEST_F(TestTxCallbackList, remove_callback_by_fast_commit) create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable3, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable1, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_100 /*scn*/); - + int sync_cnt = 9; + callback_list_.sync_log_succ(scn_100, sync_cnt); EXPECT_EQ(9, callback_list_.get_length()); fast_commit_reserve_cnt_ = 16; - bool has_remove = false; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + has_remove = false; + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(8, callback_list_.get_length()); EXPECT_EQ(1, mgr_.get_callback_remove_for_fast_commit_count()); EXPECT_EQ(true, has_remove); fast_commit_reserve_cnt_ = 14; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(6, callback_list_.get_length()); EXPECT_EQ(3, mgr_.get_callback_remove_for_fast_commit_count()); EXPECT_EQ(true, has_remove); fast_commit_reserve_cnt_ = 1; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(4, callback_list_.get_length()); EXPECT_EQ(5, mgr_.get_callback_remove_for_fast_commit_count()); EXPECT_EQ(true, has_remove); fast_commit_reserve_cnt_ = 1; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(4, callback_list_.get_length()); EXPECT_EQ(5, mgr_.get_callback_remove_for_fast_commit_count()); EXPECT_EQ(false, has_remove); @@ -469,61 +444,101 @@ TEST_F(TestTxCallbackList, remove_callback_by_rollback_to) auto savepoint0 = get_seq_no(); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); auto savepoint1 = get_seq_no(); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); auto savepoint2 = get_seq_no(); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); auto savepoint3 = get_seq_no(); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); - + true /*need_submit_log*/); EXPECT_EQ(9, callback_list_.get_length()); + auto from = get_seq_no() + 1; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint3)); + from.set_branch(savepoint3.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint3, from, share::SCN::invalid_scn())); EXPECT_EQ(8, callback_list_.get_length()); EXPECT_EQ(1, mgr_.get_callback_remove_for_rollback_to_count()); - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint2)); + from.set_branch(savepoint2.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint2, from, share::SCN::invalid_scn())); EXPECT_EQ(7, callback_list_.get_length()); EXPECT_EQ(2, mgr_.get_callback_remove_for_rollback_to_count()); - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint1)); + from.set_branch(savepoint1.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint1, from, share::SCN::invalid_scn())); EXPECT_EQ(3, callback_list_.get_length()); EXPECT_EQ(6, mgr_.get_callback_remove_for_rollback_to_count()); - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint0)); + from.set_branch(savepoint0.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint0, from, share::SCN::invalid_scn())); EXPECT_EQ(0, callback_list_.get_length()); EXPECT_EQ(9, mgr_.get_callback_remove_for_rollback_to_count()); EXPECT_EQ(9, rollback_cnt_); } +TEST_F(TestTxCallbackList, remove_callback_by_branch_rollback_to) +{ + ObMemtable *memtable1 = create_memtable(); + share::SCN scn; + scn.convert_for_logservice(1); +#define SP(branch, i) auto sp_##i = get_seq_no(); sp_##i.set_branch(branch); +#define APPEND_CB(branch, i) \ + seq_counter_.set_branch(branch); \ + auto cb_##i = create_and_append_callback(memtable1, \ + false, /*need_submit_log*/ \ + share::SCN::plus(scn, i)/*scn*/); + SP(0, 0); + APPEND_CB(1,1); + SP(1, 1); + APPEND_CB(2,2); + SP(2, 2); + APPEND_CB(0,3); + APPEND_CB(1,4); + APPEND_CB(1,5); + APPEND_CB(2,6); + seq_counter_.set_branch(0); +#undef APPEND_CB +#undef SP +#define RB_TO(i) { \ + auto from = (seq_counter_ + 1); \ + from.set_branch(sp_##i.get_branch()); \ + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(sp_##i, from, share::SCN::invalid_scn())); \ + } + EXPECT_EQ(6, callback_list_.get_length()); + + RB_TO(2); // rollback branch 2 + EXPECT_EQ(5, callback_list_.get_length()); + EXPECT_EQ(1, mgr_.get_callback_remove_for_rollback_to_count()); + + RB_TO(1); // rollback branch 1 + EXPECT_EQ(3, callback_list_.get_length()); + EXPECT_EQ(3, mgr_.get_callback_remove_for_rollback_to_count()); + + RB_TO(0); // rollback to head + EXPECT_EQ(0, callback_list_.get_length()); + EXPECT_EQ(6, mgr_.get_callback_remove_for_rollback_to_count()); + + EXPECT_EQ(6, rollback_cnt_); +} + TEST_F(TestTxCallbackList, remove_callback_by_clean_unlog_callbacks) { ObMemtable *memtable1 = create_memtable(); @@ -538,36 +553,27 @@ TEST_F(TestTxCallbackList, remove_callback_by_clean_unlog_callbacks) create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); EXPECT_EQ(9, callback_list_.get_length()); int64_t removed_cnt = 0; @@ -594,44 +600,35 @@ TEST_F(TestTxCallbackList, remove_callback_by_replay_fail) create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_4/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_4/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_4/*scn*/); EXPECT_EQ(9, callback_list_.get_length()); - EXPECT_EQ(OB_SUCCESS, callback_list_.replay_fail(scn_4 /*log_timestamp*/)); + EXPECT_EQ(OB_SUCCESS, callback_list_.replay_fail(scn_4 /*log_timestamp*/, true)); EXPECT_EQ(6, callback_list_.get_length()); EXPECT_EQ(3, rollback_cnt_); @@ -643,14 +640,11 @@ TEST_F(TestTxCallbackList, checksum_leader_tx_end_basic) ObMemtable *memtable = create_memtable(); create_and_append_callback(memtable, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); EXPECT_EQ(3, callback_list_.get_length()); @@ -672,15 +666,12 @@ TEST_F(TestTxCallbackList, checksum_follower_tx_end) create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3 /*scn*/); EXPECT_EQ(3, callback_list_.get_length()); @@ -702,22 +693,17 @@ TEST_F(TestTxCallbackList, checksum_leader_tx_end_harder) create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2 /*scn*/); create_and_append_callback(memtable, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable, - false, /*need_submit_log*/ - false /*need_fill_redo*/); + false /*need_submit_log*/); EXPECT_EQ(5, callback_list_.get_length()); @@ -738,26 +724,20 @@ TEST_F(TestTxCallbackList, checksum_leader_tx_end_harderer) create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2 /*scn*/); create_and_append_callback(memtable, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1 /*scn*/); create_and_append_callback(memtable, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable, - false, /*need_submit_log*/ - false /*need_fill_redo*/); + false /*need_submit_log*/); EXPECT_EQ(6, callback_list_.get_length()); @@ -782,36 +762,27 @@ TEST_F(TestTxCallbackList, checksum_remove_memtable_and_tx_end) create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable3, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable1, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); EXPECT_EQ(9, callback_list_.get_length()); @@ -849,57 +820,48 @@ TEST_F(TestTxCallbackList, checksum_fast_commit_and_tx_end) create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable3, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable1, - false, /*need_submit_log*/ - true /*need_fill_redo*/); + false /*need_submit_log*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); - + true /*need_submit_log*/); + int sync_cnt = 0; + callback_list_.sync_log_succ(share::SCN::plus(scn_3, 100), sync_cnt); EXPECT_EQ(9, callback_list_.get_length()); fast_commit_reserve_cnt_ = 16; - bool has_remove = false; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(true, is_checksum_equal(1, checksum_)); EXPECT_EQ(scn_2, callback_list_.checksum_scn_); fast_commit_reserve_cnt_ = 14; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(true, is_checksum_equal(3, checksum_)); EXPECT_EQ(scn_3, callback_list_.checksum_scn_); fast_commit_reserve_cnt_ = 1; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(true, is_checksum_equal(5, checksum_)); EXPECT_EQ(scn_4, callback_list_.checksum_scn_); fast_commit_reserve_cnt_ = 1; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(true, is_checksum_equal(5, checksum_)); EXPECT_EQ(scn_4, callback_list_.checksum_scn_); @@ -926,55 +888,51 @@ TEST_F(TestTxCallbackList, checksum_rollback_to_and_tx_end) auto savepoint0 = get_seq_no(); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_1/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); create_and_append_callback(memtable1, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_2/*scn*/); auto savepoint1 = get_seq_no(); create_and_append_callback(memtable3, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable2, false, /*need_submit_log*/ - false, /*need_fill_redo*/ scn_3/*scn*/); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); auto savepoint2 = get_seq_no(); create_and_append_callback(memtable3, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); auto savepoint3 = get_seq_no(); create_and_append_callback(memtable1, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); EXPECT_EQ(9, callback_list_.get_length()); + auto from = get_seq_no() + 1; - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint3)); + from.set_branch(savepoint3.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint3, from, share::SCN::invalid_scn())); EXPECT_EQ(true, is_checksum_equal(5, checksum_)); EXPECT_EQ(scn_4, callback_list_.checksum_scn_); - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint2)); + from.set_branch(savepoint2.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint2, from, share::SCN::invalid_scn())); EXPECT_EQ(true, is_checksum_equal(5, checksum_)); EXPECT_EQ(scn_4, callback_list_.checksum_scn_); - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint1)); + from.set_branch(savepoint1.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint1, from, share::SCN::invalid_scn())); EXPECT_EQ(true, is_checksum_equal(5, checksum_)); EXPECT_EQ(scn_4, callback_list_.checksum_scn_); - EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint0)); + from.set_branch(savepoint0.get_branch()); + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_rollback_to(savepoint0, from, share::SCN::invalid_scn())); EXPECT_EQ(true, is_checksum_equal(5, checksum_)); EXPECT_EQ(scn_4, callback_list_.checksum_scn_); @@ -1001,8 +959,7 @@ TEST_F(TestTxCallbackList, checksum_all_and_tx_end_test) { for (int i = 0; i < 10; i++) { ObMemtable *mt = mts[ObRandom::rand(0, mt_cnt - 1)]; create_and_append_callback(mt, - true, /*need_submit_log*/ - true /*need_fill_redo*/); + true /*need_submit_log*/); } return true; @@ -1019,11 +976,12 @@ TEST_F(TestTxCallbackList, checksum_all_and_tx_end_test) { it = it->next_) { i++; it->need_submit_log_ = false; - it->need_fill_redo_ = false; it->scn_.convert_for_logservice(cur_log); enable = true; need_submit_head = it; my_calculate.add_bit(it->get_seq_no().get_seq()); + int64_t c = 1; + callback_list_.sync_log_succ(it->scn_, c); } if (!enable) { @@ -1042,7 +1000,7 @@ TEST_F(TestTxCallbackList, checksum_all_and_tx_end_test) { for (ObITransCallback* it = need_submit_head->next_; it != &(callback_list_.head_); it = it->next_) { - if (it->need_submit_log_ || it->need_fill_redo_) { + if (it->need_submit_log_) { break; } else if (it->get_memtable() == mt) { enable = true; @@ -1071,10 +1029,10 @@ TEST_F(TestTxCallbackList, checksum_all_and_tx_end_test) { enable = true; } - bool has_remove = false; + has_remove = false; if (enable) { EXPECT_EQ(OB_SUCCESS, - callback_list_.remove_callbacks_for_fast_commit(nullptr, has_remove)); + callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(true, has_remove); } @@ -1091,12 +1049,17 @@ TEST_F(TestTxCallbackList, checksum_all_and_tx_end_test) { auto seq = from + ObRandom::rand(1, range_cnt - 1); enable = true; if (enable) { + bool reset_need_submit_head = false; if (need_submit_head->get_seq_no() > seq) { + reset_need_submit_head = true; + } + auto from0 = get_seq_no() + 1; + EXPECT_EQ(OB_SUCCESS, + callback_list_.remove_callbacks_for_rollback_to(seq, from0, share::SCN::invalid_scn())); + EXPECT_EQ(false, callback_list_.empty()); + if (reset_need_submit_head) { need_submit_head = callback_list_.head_.prev_; } - EXPECT_EQ(OB_SUCCESS, - callback_list_.remove_callbacks_for_rollback_to(seq)); - EXPECT_EQ(false, callback_list_.empty()); } } @@ -1131,15 +1094,183 @@ TEST_F(TestTxCallbackList, checksum_all_and_tx_end_test) { it != &(callback_list_.head_); it = it->next_) { EXPECT_EQ(it->need_submit_log_, true); - EXPECT_EQ(it->need_fill_redo_, true); - my_calculate.add_bit(it->get_seq_no().get_seq()); + // set scn because tx_commit will do check + it->scn_.convert_for_logservice(10000); + //my_calculate.add_bit(it->get_seq_no().get_seq()); } - + // calc checksum of remains by fast_commit + fast_commit_reserve_cnt_ = 0; + EXPECT_EQ(OB_SUCCESS, callback_list_.remove_callbacks_for_fast_commit()); EXPECT_EQ(OB_SUCCESS, callback_list_.tx_commit()); EXPECT_EQ(true, my_calculate.equal(checksum_)); } +TEST_F(TestTxCallbackList, log_cursor) { + TRANS_LOG(INFO, "CASE: log_cursor"); + int ret = 0; + auto memtable1 = create_memtable(); + share::SCN scn; scn.convert_for_logservice(100); + ObTxFillRedoCtx ctx; + memtable::ObRedoLogSubmitHelper helper; + ctx.helper_ = &helper; + struct ObTxCallbackFunctorAdapter : public ObITxFillRedoFunctor { + ObTxCallbackFunctorAdapter(ObFunction func) : func_(func) {} + int operator()(ObITransCallback*cb) { return func_(cb); } + ObFunction func_; + }; +#define APPEND_CB(branch, i) \ + seq_counter_.set_branch(branch); \ + auto cb_##i = create_and_append_callback(memtable1, \ + true /*need_submit_log*/); \ + +#define LOG_SUBMITED(SCOPE_X) \ + { \ + int submitted_cnt = 0; \ + ObArrayHelper scope_a(1, &SCOPE_X, 1); \ + scn = share::SCN::plus(scn, 1); \ + ret = mgr_.log_submitted(scope_a, scn, submitted_cnt); \ + } + +#define LOG_SYNC_SUCC(SCOPE_X) \ + { \ + ObArrayHelper scope_a(1, &SCOPE_X, 1); \ + int64_t cnt = 0; \ + ret = mgr_.log_sync_succ(scope_a, scn, cnt); \ + } + + // check log_cursor is valid and expected after every op on CallbackList + // . init callback list, insert 3 node + APPEND_CB(0, 1); + APPEND_CB(0, 2); + APPEND_CB(0, 3); + EXPECT_EQ(callback_list_.log_cursor_, cb_1); + EXPECT_TRUE(cb_3->need_submit_log_); + // . fill log for 1 and 2, log_cursor point to head + int i = 1; + ObCallbackScope scope; + ObTxCallbackFunctorAdapter f([&](ObITransCallback*cb) -> int { + if (*scope.start_ == NULL) scope.start_ = cb; + scope.end_ = cb; + scope.host_ = &callback_list_; + scope.cnt_ = i; + return i++ == 2 ? OB_EAGAIN : OB_SUCCESS; + }); + ctx.callback_scope_ = &scope; + ret = callback_list_.fill_log(callback_list_.log_cursor_, ctx, f); + EXPECT_EQ(OB_EAGAIN, ret); + EXPECT_EQ(i, 3); + EXPECT_EQ(cb_1, *scope.start_); + EXPECT_EQ(cb_2, *scope.end_); + EXPECT_TRUE(cb_3->need_submit_log_); + EXPECT_EQ(callback_list_.log_cursor_, cb_1); + // . log submitted, update log_cursor, point to the next one to log + LOG_SUBMITED(scope); + EXPECT_EQ(OB_SUCCESS, ret); + EXPECT_EQ(callback_list_.log_cursor_, cb_3); + // . log_sync succ, fast_commit remove callbacks, log cursor is not affected + LOG_SYNC_SUCC(scope); + EXPECT_EQ(OB_SUCCESS, ret); + has_remove = false; + ret = callback_list_.remove_callbacks_for_fast_commit(); + EXPECT_EQ(OB_SUCCESS, ret); + EXPECT_TRUE(has_remove); + EXPECT_EQ(1, callback_list_.get_length()); + EXPECT_EQ(callback_list_.log_cursor_, cb_3); + // . fill redo, check continous : [1,2](logged) -> [3](will do log) + scope.reset(); + i = 0; + { + ObTxCallbackFunctorAdapter f([&](ObITransCallback*cb) -> int { + if (*scope.start_ == NULL) scope.start_ = cb; + scope.end_ = cb; + scope.host_ = &callback_list_; + scope.cnt_ = ++i; + return OB_SUCCESS; + }); + ret = callback_list_.fill_log(callback_list_.log_cursor_, ctx, f); + } + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_FALSE(scope.is_empty()); + EXPECT_EQ(*scope.start_, *scope.end_); + EXPECT_EQ(*scope.start_, cb_3); + // . log submitted, udpate log_cursor: because no other log need to be log, + // log cursor point to head + LOG_SUBMITED(scope); + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_EQ(callback_list_.log_cursor_, &callback_list_.head_); + // . log_sync fail, because failed callbacks are removed, log cursor is unchanged + int64_t removed_cnt = 0; + ret = callback_list_.sync_log_fail(scope, scn, removed_cnt); + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_EQ(1, removed_cnt); + EXPECT_EQ(callback_list_.log_cursor_, &callback_list_.head_); + // . fill log will return empty, because of no callbacks need to log + scope.reset(); + i = 0; + { + ObTxCallbackFunctorAdapter f([&](ObITransCallback*cb) -> int { + if (*scope.start_ == NULL) scope.start_ = cb; + scope.end_ = cb; + scope.host_ = &callback_list_; + scope.cnt_ = ++i; + return OB_SUCCESS; + }); + ret = callback_list_.fill_log(callback_list_.log_cursor_, ctx, f); + } + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_TRUE(scope.is_empty()); + EXPECT_EQ(callback_list_.log_cursor_, &callback_list_.head_); + // + // TEST rollback savepoint affect log_cursor + // + // . do append 5 callback, logging out 2, log_cursor point to 3th, rollback savepoint to 1 + // the log_cusor should be adjust to the 1st.next_ + // + // insert callbacks: 4, 5, then flush the log, log cursor point to head + APPEND_CB(0, 4); + EXPECT_EQ(callback_list_.log_cursor_, cb_4); + auto sp0 = get_seq_no(); + APPEND_CB(0, 5); + EXPECT_EQ(callback_list_.log_cursor_, cb_4); + scope.reset(); + i = 0; + { + ObTxCallbackFunctorAdapter f([&](ObITransCallback*cb) -> int { + if (*scope.start_ == NULL) scope.start_ = cb; + scope.end_ = cb; + scope.host_ = &callback_list_; + scope.cnt_ = ++i; + return OB_SUCCESS; + }); + ret = callback_list_.fill_log(callback_list_.log_cursor_, ctx, f); + } + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_EQ(callback_list_.log_cursor_, cb_4); + // . all log submitted, update log_cursor, point to head + LOG_SUBMITED(scope); + EXPECT_EQ(callback_list_.log_cursor_, &callback_list_.head_); + // append new callbacks 6,7, the log cursor should point to 6 + auto sp1= get_seq_no(); sp1.set_branch(1); + APPEND_CB(1, 6); + EXPECT_EQ(callback_list_.log_cursor_, cb_6); + APPEND_CB(0, 7); + APPEND_CB(0, 8); + EXPECT_EQ(callback_list_.log_cursor_, cb_6); + // must logging synced or failed before do rollback + LOG_SYNC_SUCC(scope); + EXPECT_EQ(ret, OB_SUCCESS); + auto from = get_seq_no() + 1; + from.set_branch(sp1.get_branch()); + ret = callback_list_.remove_callbacks_for_rollback_to(sp1, from, share::SCN::invalid_scn()); + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_EQ(callback_list_.log_cursor_, cb_7); + from.set_branch(sp0.get_branch()); + ret = callback_list_.remove_callbacks_for_rollback_to(sp0, from, share::SCN::invalid_scn()); + EXPECT_EQ(ret, OB_SUCCESS); + EXPECT_EQ(callback_list_.log_cursor_, cb_4->next_); +#undef APPEND_CB +} } // namespace unittest namespace memtable @@ -1158,27 +1289,24 @@ void ObMemtableCtx::callback_free(ObITransCallback *cb) } } -int ObTxCallbackList::remove_callbacks_for_fast_commit(const ObITransCallback *callback, - bool &has_remove) +int ObTxCallbackList::remove_callbacks_for_fast_commit(const share::SCN stop_scn) { int ret = OB_SUCCESS; - has_remove = false; - ObByteLockGuard guard(latch_); - + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); const int64_t fast_commit_callback_count = unittest::TestTxCallbackList::fast_commit_reserve_cnt_; const int64_t recommand_reserve_count = (fast_commit_callback_count + 1) / 2; const int64_t need_remove_count = length_ - recommand_reserve_count; - ObRemoveCallbacksForFastCommitFunctor functor(callback, need_remove_count); + ObRemoveCallbacksForFastCommitFunctor functor(need_remove_count, sync_scn_); functor.set_checksumer(checksum_scn_, &batch_checksum_); - if (OB_FAIL(callback_(functor))) { + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(ERROR, "remove callbacks for fast commit wont report error", K(ret), K(functor)); } else { callback_mgr_.add_fast_commit_callback_remove_cnt(functor.get_remove_cnt()); ensure_checksum_(functor.get_checksum_last_scn()); - has_remove = share::SCN::min_scn() != functor.get_checksum_last_scn(); - if (has_remove) { + unittest::has_remove = share::SCN::min_scn() != functor.get_checksum_last_scn(); + if (unittest::has_remove) { TRANS_LOG(INFO, "remove callbacks for fast commit", K(functor), K(*this)); } } @@ -1191,14 +1319,15 @@ int ObTxCallbackList::remove_callbacks_for_remove_memtable( const share::SCN) { int ret = OB_SUCCESS; - ObByteLockGuard guard(latch_); + LockGuard guard(*this, LOCK_MODE::LOCK_ITERATE); - ObRemoveSyncCallbacksWCondFunctor functor( - // condition for remove - [memtable_set](ObITransCallback *callback) -> bool { + struct Functor : public ObRemoveSyncCallbacksWCondFunctor { + Functor(const bool need_remove_data = true, const bool is_reverse = false) + : ObRemoveSyncCallbacksWCondFunctor(need_remove_data, is_reverse) {} + bool cond_for_remove(ObITransCallback *callback) { int ret = OB_SUCCESS; int bool_ret = true; - if (OB_HASH_EXIST == (ret = memtable_set->exist_refactored((uint64_t)callback->get_memtable()))) { + if (OB_HASH_EXIST == (ret = memtable_set_->exist_refactored((uint64_t)callback->get_memtable()))) { bool_ret = true; } else if (OB_HASH_NOT_EXIST == ret) { bool_ret = false; @@ -1207,14 +1336,16 @@ int ObTxCallbackList::remove_callbacks_for_remove_memtable( ob_abort(); } return bool_ret; - }, // condition for stop - [](ObITransCallback *) -> bool { + } + bool cond_for_stop(ObITransCallback *) const { return false; - }, - false /*need_remove_data*/); + } + const memtable::ObMemtableSet *memtable_set_; + } functor(false /*need_remove_data*/); + functor.memtable_set_ = memtable_set; functor.set_checksumer(checksum_scn_, &batch_checksum_); - if (OB_FAIL(callback_(functor))) { + if (OB_FAIL(callback_(functor, guard.state_))) { TRANS_LOG(ERROR, "remove callbacks for remove memtable wont report error", K(ret), K(functor)); } else { callback_mgr_.add_release_memtable_callback_remove_cnt(functor.get_remove_cnt()); diff --git a/unittest/storage/mock_ob_log_handler.h b/unittest/storage/mock_ob_log_handler.h index 7ce9113d01..9b44353d1d 100644 --- a/unittest/storage/mock_ob_log_handler.h +++ b/unittest/storage/mock_ob_log_handler.h @@ -469,6 +469,11 @@ public: scn.set_max(); return OB_SUCCESS; } + int get_max_decided_scn_as_leader(share::SCN &scn) const + { + scn.set_max(); + return OB_SUCCESS; + } int get_max_decided_log_ts_ns(int64_t &log_ts) { log_ts = INT64_MAX; diff --git a/unittest/storage/test_compaction_policy.cpp b/unittest/storage/test_compaction_policy.cpp index 78767a9edd..32b69e7ca4 100644 --- a/unittest/storage/test_compaction_policy.cpp +++ b/unittest/storage/test_compaction_policy.cpp @@ -367,10 +367,10 @@ int TestCompactionPolicy::mock_memtable( snapshot_scn.convert_for_tx(snapshot_version); memtable->snapshot_version_ = snapshot_scn; memtable->write_ref_cnt_ = 0; - memtable->unsynced_cnt_ = 0; + memtable->unsubmitted_cnt_ = 0; memtable->is_tablet_freeze_ = true; memtable->state_ = ObMemtableState::MINOR_FROZEN; - memtable->set_resolve_active_memtable_left_boundary(true); + memtable->set_resolved_active_memtable_left_boundary(true); memtable->set_frozen(); memtable->location_ = storage::checkpoint::ObFreezeCheckpointLocation::PREPARE; } diff --git a/unittest/storage/tx/CMakeLists.txt b/unittest/storage/tx/CMakeLists.txt index 4ffe346ea6..ed28004b2e 100644 --- a/unittest/storage/tx/CMakeLists.txt +++ b/unittest/storage/tx/CMakeLists.txt @@ -40,6 +40,7 @@ storage_unittest(test_ob_tx_log) storage_unittest(test_ob_timestamp_service) storage_unittest(test_ob_trans_rpc) storage_unittest(test_ob_tx_msg) +storage_unittest(test_undo_action) storage_unittest(test_ob_id_meta) storage_unittest(test_ob_standby_read) storage_unittest(test_ob_standby_read_transfer) @@ -47,3 +48,7 @@ storage_unittest(test_ob_trans_tlog) add_subdirectory(it) storage_unittest(test_dup_table_lease) + +storage_unittest(test_redo_submitter) +storage_unittest(test_trans_callback_mgr_fill_redo) +storage_unittest(test_misc) diff --git a/unittest/storage/tx/it/test_tx.cpp b/unittest/storage/tx/it/test_tx.cpp index 3d04b1609e..c5eaec2e24 100644 --- a/unittest/storage/tx/it/test_tx.cpp +++ b/unittest/storage/tx/it/test_tx.cpp @@ -76,7 +76,6 @@ int check_sequence_set_violation(const concurrent_control::ObWriteFlag , return OB_SUCCESS; } } - class ObTestTx : public ::testing::Test { public: @@ -84,8 +83,8 @@ public: { oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION); ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001); - const uint64_t tv = ObTimeUtility::current_time(); - ObCurTraceId::set(&tv); + ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1",2023); + ObCurTraceId::init(ip_port); GCONF._ob_trans_rpc_timeout = 500; ObClockGenerator::init(); const testing::TestInfo* const test_info = @@ -2414,6 +2413,51 @@ TEST_F(ObTestTx, interrupt_get_read_snapshot) ROLLBACK_TX(n1, tx); } +TEST_F(ObTestTx, rollback_with_branch_savepoint) +{ + START_ONE_TX_NODE(n1); + PREPARE_TX(n1, tx); + PREPARE_TX_PARAM(tx_param); + CREATE_IMPLICIT_SAVEPOINT(n1, tx, tx_param, global_sp1); + CREATE_BRANCH_SAVEPOINT(n1, tx, 100, sp_b100_1); + ASSERT_EQ(OB_SUCCESS, n1->write(tx, 100, 111, 100)); + CREATE_BRANCH_SAVEPOINT(n1, tx, 200, sp_b200_1); + ASSERT_EQ(OB_SUCCESS, n1->write(tx, 200, 211, 200)); + ASSERT_EQ(OB_SUCCESS, n1->write(tx, 101, 112, 100)); + ASSERT_EQ(OB_SUCCESS, n1->write(tx, 500, 505)); // global write + ASSERT_EQ(OB_SUCCESS, n1->write(tx, 201, 212, 200)); + // rollback branch 200 + ASSERT_EQ(OB_SUCCESS, ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, sp_b200_1, 2000*1000)); + // check branch 100 is readable + int64_t val = 0; + ASSERT_EQ(OB_SUCCESS, n1->read(tx, 101, val)); + ASSERT_EQ(val, 112); + // check global write is readable + ASSERT_EQ(OB_SUCCESS, n1->read(tx, 500, val)); + ASSERT_EQ(val, 505); + // check branch 200 is un-readable + ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 200, val)); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 201, val)); + // write with branch 200 + ASSERT_EQ(OB_SUCCESS, n1->write(tx, 206, 602, 200)); + // rollback branch 100 + ASSERT_EQ(OB_SUCCESS, ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, sp_b100_1, 2000*1000)); + // check global write is readable + ASSERT_EQ(OB_SUCCESS, n1->read(tx, 500, val)); + ASSERT_EQ(val, 505); + // check branch 200 is readable + ASSERT_EQ(OB_SUCCESS, n1->read(tx, 206, val)); + ASSERT_EQ(val, 602); + // check branch 100 is un-readable + ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 100, val)); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 101, val)); + // rollback global + ASSERT_EQ(OB_SUCCESS, ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, global_sp1, 2000 * 1000)); + // check global and branch 200 is un-readable + ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 500, val)); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 206, val)); + ROLLBACK_TX(n1, tx); +} //// /// APPEND NEW TEST HERE, USE PRE DEFINED MACRO IN FILE `test_tx.dsl` /// SEE EXAMPLE: TEST_F(ObTestTx, rollback_savepoint_timeout) @@ -2423,6 +2467,10 @@ TEST_F(ObTestTx, interrupt_get_read_snapshot) int main(int argc, char **argv) { + uint64_t checksum = 1100101; + uint64_t c = 0; + uint64_t checksum1 = ob_crc64(checksum, (void*)&c, sizeof(uint64_t)); + uint64_t checksum2 = ob_crc64(c, (void*)&checksum, sizeof(uint64_t)); int64_t tx_id = 21533427; uint64_t h = murmurhash(&tx_id, sizeof(tx_id), 0); system("rm -rf test_tx.log*"); @@ -2433,6 +2481,6 @@ int main(int argc, char **argv) "test_tx.log"); // audit logger.set_log_level(OB_LOG_LEVEL_DEBUG); ::testing::InitGoogleTest(&argc, argv); - TRANS_LOG(INFO, "mmhash:", K(h)); + TRANS_LOG(INFO, "mmhash:", K(h), K(checksum1), K(checksum2)); return RUN_ALL_TESTS(); } diff --git a/unittest/storage/tx/it/test_tx_ctx.cpp b/unittest/storage/tx/it/test_tx_ctx.cpp index 0c99cca399..c9d2f57ff4 100644 --- a/unittest/storage/tx/it/test_tx_ctx.cpp +++ b/unittest/storage/tx/it/test_tx_ctx.cpp @@ -130,7 +130,7 @@ TEST_F(ObTestTxCtx, DelayAbort) ASSERT_EQ(OB_SUCCESS, ls_tx_ctx_mgr->get_tx_ctx(tx.tx_id_, false /*for_replay*/, tx_ctx)); GCONF._private_buffer_size = 1; // ASSERT_EQ(OB_SUCCESS, tx_ctx->submit_log_impl_(ObTxLogType::TX_REDO_LOG)); - ASSERT_EQ(OB_SUCCESS, tx_ctx->submit_redo_log(false /*is_freeze*/)); + ASSERT_EQ(OB_SUCCESS, tx_ctx->submit_redo_after_write(false, ObTxSEQ())); TRANS_LOG(INFO, "[TEST] after submit redo", K(tx_ctx->trans_id_), K(tx_ctx->exec_info_.max_applied_log_ts_)); n1->wait_all_redolog_applied(); diff --git a/unittest/storage/tx/it/test_tx_dsl.h b/unittest/storage/tx/it/test_tx_dsl.h index 750dbc4b73..af56d34ac1 100644 --- a/unittest/storage/tx/it/test_tx_dsl.h +++ b/unittest/storage/tx/it/test_tx_dsl.h @@ -53,8 +53,12 @@ ObTxSEQ sp; \ ASSERT_EQ(OB_SUCCESS, n1->create_implicit_savepoint(tx, tx_param, sp)); +#define CREATE_BRANCH_SAVEPOINT(n1, tx, branch, sp) \ + ObTxSEQ sp; \ + ASSERT_EQ(OB_SUCCESS, n1->create_branch_savepoint(tx, branch, sp)); + #define ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, sp, timeout_us) \ - n1->rollback_to_implicit_savepoint(tx, sp, n1->ts_after_us(timeout_us), nullptr); + n1->rollback_to_implicit_savepoint(tx, sp, n1->ts_after_us(timeout_us), nullptr) #define INJECT_LINK_FAILURE(n1, n2) \ ASSERT_EQ(OB_SUCCESS, bus_.inject_link_failure(n1->addr_, n2->addr_)); \ diff --git a/unittest/storage/tx/it/test_tx_free_route.cpp b/unittest/storage/tx/it/test_tx_free_route.cpp index df1875e124..bc79fe56d4 100644 --- a/unittest/storage/tx/it/test_tx_free_route.cpp +++ b/unittest/storage/tx/it/test_tx_free_route.cpp @@ -648,8 +648,8 @@ public: { oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION); ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001); - const uint64_t tv = ObTimeUtility::current_time(); - ObCurTraceId::set(&tv); + ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1", 2023); + ObCurTraceId::init(ip_port); GCONF._ob_trans_rpc_timeout = 500; ObClockGenerator::init(); omt::the_ctrl_of_enable_transaction_free_route = true; diff --git a/unittest/storage/tx/it/tx_node.cpp b/unittest/storage/tx/it/tx_node.cpp index 2dd2316e43..0777815dba 100644 --- a/unittest/storage/tx/it/tx_node.cpp +++ b/unittest/storage/tx/it/tx_node.cpp @@ -22,6 +22,13 @@ } while(0); namespace oceanbase { +namespace common { +int ObClusterVersion::get_tenant_data_version(const uint64_t tenant_id, uint64_t &data_version) +{ + data_version = DATA_CURRENT_VERSION; + return OB_SUCCESS; +} +} namespace share { void* ObMemstoreAllocator::alloc(AllocHandle& handle, int64_t size, const int64_t expire_ts) @@ -152,6 +159,7 @@ int ObTxNode::start() { fake_tx_log_adapter_ = new ObFakeTxLogAdapter(); OZ(fake_tx_log_adapter_->start()); } + get_ts_mgr_().reset(); OZ(msg_consumer_.start()); OZ(txs_.start()); OZ(create_ls_(ls_id_)); @@ -243,6 +251,12 @@ ObTxNode::~ObTxNode() __attribute__((optnone)) { ObTenantEnv::set_tenant(&tenant_); OZ(txs_.tx_ctx_mgr_.revert_ls_tx_ctx_mgr(fake_tx_table_.tx_ctx_table_.ls_tx_ctx_mgr_)); fake_tx_table_.tx_ctx_table_.ls_tx_ctx_mgr_ = nullptr; + bool is_tx_clean = false; + int retry_cnt = 0; + do { + usleep(2000); + txs_.block_tx(ls_id_, is_tx_clean); + } while(!is_tx_clean && ++retry_cnt < 1000); OX(txs_.stop()); OZ(txs_.wait_()); if (role_ == Leader && fake_tx_log_adapter_) { @@ -561,7 +575,7 @@ int ObTxNode::atomic_write(ObTxDesc &tx, const int64_t key, const int64_t value, } return ret; } -int ObTxNode::write(ObTxDesc &tx, const int64_t key, const int64_t value) +int ObTxNode::write(ObTxDesc &tx, const int64_t key, const int64_t value, const int16_t branch) { int ret = OB_SUCCESS; ObTxReadSnapshot snapshot; @@ -569,13 +583,14 @@ int ObTxNode::write(ObTxDesc &tx, const int64_t key, const int64_t value) tx.isolation_, ts_after_ms(50), snapshot)); - OZ(write(tx, snapshot, key, value)); + OZ(write(tx, snapshot, key, value, branch)); return ret; } int ObTxNode::write(ObTxDesc &tx, const ObTxReadSnapshot &snapshot, const int64_t key, - const int64_t value) + const int64_t value, + const int16_t branch) { TRANS_LOG(INFO, "write", K(key), K(value), K(snapshot), K(tx), KPC(this)); int ret = OB_SUCCESS; @@ -589,6 +604,7 @@ int ObTxNode::write(ObTxDesc &tx, write_store_ctx.ls_ = &mock_ls_; write_store_ctx.ls_id_ = ls_id_; write_store_ctx.table_iter_ = iter; + write_store_ctx.branch_ = branch; concurrent_control::ObWriteFlag write_flag; OZ(txs_.get_write_store_ctx(tx, snapshot, @@ -709,20 +725,28 @@ int ObTxNode::replay(const void *buffer, const int64_t ts_ns) { ObTenantEnv::set_tenant(&tenant_); - int ret = OB_SUCCESS; logservice::ObLogBaseHeader base_header; int64_t tmp_pos = 0; const char *log_buf = static_cast(buffer); - if (OB_FAIL(base_header.deserialize(log_buf, nbytes, tmp_pos))) { LOG_WARN("log base header deserialize error", K(ret)); - } else if (OB_FAIL(ObFakeTxReplayExecutor::execute(&mock_ls_, mock_ls_.get_tx_svr(), log_buf, nbytes, - tmp_pos, lsn, ts_ns, base_header.get_replay_hint(), - ls_id_, tenant_id_, memtable_))) { - LOG_WARN("replay tx log error", K(ret), K(lsn), K(ts_ns)); } else { - LOG_INFO("replay tx log succ", K(ret), K(lsn), K(ts_ns)); + share::SCN log_scn; + log_scn.convert_for_tx(ts_ns); + ObFakeTxReplayExecutor executor(&mock_ls_, + ls_id_, + tenant_id_, + mock_ls_.get_tx_svr(), + lsn, + log_scn, + base_header); + executor.set_memtable(memtable_); + if (OB_FAIL(executor.execute(log_buf, nbytes, tmp_pos))) { + LOG_WARN("replay tx log error", K(ret), K(lsn), K(ts_ns)); + } else { + LOG_INFO("replay tx log succ", K(ret), K(lsn), K(ts_ns)); + } } return ret; } diff --git a/unittest/storage/tx/it/tx_node.h b/unittest/storage/tx/it/tx_node.h index 06b5bc6f85..8d51216072 100644 --- a/unittest/storage/tx/it/tx_node.h +++ b/unittest/storage/tx/it/tx_node.h @@ -138,11 +138,12 @@ public: int read(const ObTxReadSnapshot &snapshot, const int64_t key, int64_t &value); - int write(ObTxDesc &tx, const int64_t key, const int64_t value); + int write(ObTxDesc &tx, const int64_t key, const int64_t value, const int16_t branch = 0); int write(ObTxDesc &tx, const ObTxReadSnapshot &snapshot, const int64_t key, - const int64_t value); + const int64_t value, + const int16_t branch = 0); int atomic_write(ObTxDesc &tx, const int64_t key, const int64_t value, const int64_t expire_ts, const ObTxParam &tx_param); int replay(const void *buffer, const int64_t nbytes, const palf::LSN &lsn, const int64_t ts_ns); @@ -167,6 +168,7 @@ public: DELEGATE_TENANT_WITH_RET(txs_, abort_tx, int); DELEGATE_TENANT_WITH_RET(txs_, submit_commit_tx, int); DELEGATE_TENANT_WITH_RET(txs_, get_read_snapshot, int); + DELEGATE_TENANT_WITH_RET(txs_, create_branch_savepoint, int); DELEGATE_TENANT_WITH_RET(txs_, create_implicit_savepoint, int); DELEGATE_TENANT_WITH_RET(txs_, create_explicit_savepoint, int); DELEGATE_TENANT_WITH_RET(txs_, rollback_to_explicit_savepoint, int); diff --git a/unittest/storage/tx/mock_utils/basic_fake_define.h b/unittest/storage/tx/mock_utils/basic_fake_define.h index 7e6becb37f..51097a372f 100644 --- a/unittest/storage/tx/mock_utils/basic_fake_define.h +++ b/unittest/storage/tx/mock_utils/basic_fake_define.h @@ -242,6 +242,11 @@ public: get_gts_error_ = OB_SUCCESS; } public: + void reset() { + get_gts_error_ = OB_SUCCESS; + elapse_waiting_mode_ = false; + get_gts_waiting_mode_ = false; + } int update_gts(const uint64_t tenant_id, const int64_t gts, bool &update) { return OB_SUCCESS; } int get_gts(const uint64_t tenant_id, const MonotonicTs stc, @@ -262,7 +267,7 @@ public: ret = OB_EAGAIN; } } - TRANS_LOG(INFO, "get gts end", K(ret), K(gts_), K(gts), K(>s_)); + TRANS_LOG(INFO, "get gts end", K(ret), K(gts_), K(gts), K(get_gts_waiting_mode_)); return ret; } @@ -402,6 +407,7 @@ public: ObSpScLinkQueue apply_task_queue_arr[TASK_QUEUE_CNT]; ObSpScLinkQueue replay_task_queue_arr[TASK_QUEUE_CNT]; share::SCN max_submit_scn_ = share::SCN::invalid_scn(); + share::SCN max_committed_scn_ = share::SCN::invalid_scn(); void run1() { while(true) { @@ -413,6 +419,7 @@ public: ObLink *task = apply_task_queue_arr[i].pop(); if (task) { ++process_cnt; + max_committed_scn_ = static_cast(task)->cb_->__get_scn(); static_cast(task)->cb_->on_success(); delete task; ATOMIC_DEC(&inflight_cnt_); @@ -558,7 +565,10 @@ public: } return OB_SUCCESS; } - + int get_palf_committed_max_scn(share::SCN &scn) const { + scn = max_committed_scn_; + return OB_SUCCESS; + } int get_append_mode_initial_scn(share::SCN &ref_scn) { int ret = OB_SUCCESS; ref_scn = share::SCN::invalid_scn(); @@ -603,54 +613,26 @@ class ObFakeTxReplayExecutor : public ObTxReplayExecutor { public: ObFakeTxReplayExecutor(storage::ObLS *ls, + const share::ObLSID &ls_id, + const uint64_t tenant_id, storage::ObLSTxService *ls_tx_srv, const palf::LSN &lsn, - const share::SCN &log_timestamp) - : ObTxReplayExecutor(ls, ls_tx_srv, lsn, log_timestamp) {memtable_ = nullptr;} - - ~ObFakeTxReplayExecutor() {} - - static int execute(storage::ObLS *ls, - storage::ObLSTxService *ls_tx_srv, - const char *buf, - const int64_t size, - const int skip_pos, - const palf::LSN &lsn, - const int64_t &log_timestamp, - const int64_t &replay_hint, - const share::ObLSID &ls_id, - const int64_t &tenant_id, - memtable::ObMemtable* memtable) - { - int ret = OB_SUCCESS; - share::SCN log_scn; - log_scn.convert_for_gts(log_timestamp); - ObFakeTxReplayExecutor replay_executor(ls, ls_tx_srv, lsn, log_scn); - if (OB_ISNULL(ls) || OB_ISNULL(ls_tx_srv) || OB_ISNULL(buf) || size <= 0 - || 0 >= log_timestamp || INT64_MAX == log_timestamp || !lsn.is_valid()) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(ERROR, "invaild arguments", K(replay_executor), K(buf), K(size)); - } else if (replay_executor.set_memtable(memtable)) { - } else if (OB_FAIL(replay_executor.do_replay_(buf, - size, - skip_pos, - replay_hint, - ls_id, - tenant_id))) { - TRANS_LOG(ERROR, "replay_executor.do_replay failed", - K(replay_executor), K(buf), K(size), K(skip_pos), K(replay_hint), K(ls_id), K(tenant_id)); - hex_dump(buf, size, true, OB_LOG_LEVEL_INFO); - } - return ret; - } - -private: + const share::SCN &log_timestamp, + const logservice::ObLogBaseHeader &base_header) + : ObTxReplayExecutor(ls, ls_id, tenant_id, ls_tx_srv, lsn, log_timestamp, base_header) + { memtable_ = nullptr; } + ~ObFakeTxReplayExecutor() { } int set_memtable(memtable::ObMemtable* memtable) { memtable_ = memtable; return OB_SUCCESS; } - + int execute(const char *buf, + const int64_t size, + const int skip_pos) + { + return do_replay_(buf, size, skip_pos); + } int replay_one_row_in_memtable_(memtable::ObMutatorRowHeader& row_head, memtable::ObMemtableMutatorIterator *mmi_ptr) override { @@ -659,13 +641,12 @@ private: storeCtx.ls_id_ = ctx_->get_ls_id(); storeCtx.mvcc_acc_ctx_.tx_ctx_ = ctx_; storeCtx.mvcc_acc_ctx_.mem_ctx_ = mt_ctx_; - storeCtx.replay_log_scn_ = log_ts_ns_; storeCtx.tablet_id_ = row_head.tablet_id_; storeCtx.mvcc_acc_ctx_.tx_id_ = ctx_->get_trans_id(); switch (row_head.mutator_type_) { case memtable::MutatorType::MUTATOR_ROW: { - if (OB_FAIL(memtable_->replay_row(storeCtx, mmi_ptr_))) { + if (OB_FAIL(memtable_->replay_row(storeCtx, log_ts_ns_, mmi_ptr_))) { TRANS_LOG(WARN, "[Replay Tx] replay row error", K(ret)); } else { TRANS_LOG(INFO, "[Replay Tx] replay row in memtable success"); diff --git a/unittest/storage/tx/ob_mock_tx_log_adapter.h b/unittest/storage/tx/ob_mock_tx_log_adapter.h index 391cbac498..a5f43e1f63 100644 --- a/unittest/storage/tx/ob_mock_tx_log_adapter.h +++ b/unittest/storage/tx/ob_mock_tx_log_adapter.h @@ -75,6 +75,11 @@ public: UNUSED(scn); return OB_SUCCESS; } + int get_palf_committed_max_scn(share::SCN &scn) const + { + UNUSED(scn); + return OB_SUCCESS; + } int get_append_mode_initial_scn(share::SCN &ref_scn) { int ret = OB_SUCCESS; ref_scn = share::SCN::invalid_scn(); diff --git a/unittest/storage/tx/test_ls_log_writer.cpp b/unittest/storage/tx/test_ls_log_writer.cpp index 31a36d8b0e..d85b4a7436 100644 --- a/unittest/storage/tx/test_ls_log_writer.cpp +++ b/unittest/storage/tx/test_ls_log_writer.cpp @@ -85,15 +85,12 @@ TEST_F(TestLSLogWriter, submit_start_working_log) ObTxStartWorkingLog sw_log(tmp_ref); int64_t test_leader_epoch = 1308; - ObTxLogBlockHeader block_header; - ASSERT_EQ(OB_SUCCESS, ls_log_writer.init(tmp_tenant_id, TEST_LS_ID, &tx_log_adapter, (ObLSTxCtxMgr *)&tmp_mgr)); ASSERT_EQ(OB_SUCCESS, ls_log_writer.submit_start_working_log(test_leader_epoch, log_ts)); ASSERT_EQ(true, tx_log_adapter.get_log(log_ts.get_val_for_gts(), log_string)); - ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(log_string.c_str(), log_string.size(), - replay_hint, block_header)); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(log_string.c_str(), log_string.size())); ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header)); EXPECT_EQ(ObTxLogType::TX_START_WORKING_LOG, log_header.get_tx_log_type()); ASSERT_EQ(OB_SUCCESS, replay_block.deserialize_log_body(sw_log)); diff --git a/unittest/storage/tx/test_misc.cpp b/unittest/storage/tx/test_misc.cpp new file mode 100644 index 0000000000..ef53dc71d0 --- /dev/null +++ b/unittest/storage/tx/test_misc.cpp @@ -0,0 +1,131 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define protected public +#include +#include "share/ob_errno.h" +#include "lib/oblog/ob_log.h" +#include "storage/memtable/ob_memtable_context.h" + +namespace oceanbase +{ +using namespace common; +using namespace transaction; +namespace unittest +{ + +class TestObTxMisc : public ::testing::Test +{ +public : + virtual void SetUp() {} + virtual void TearDown() {} +}; + +TEST_F(TestObTxMisc, multiple_checksum_collapse_for_commit_log) +{ + TRANS_LOG(INFO, "called", "func", test_info_->name()); + // only one checksum + { + uint64_t checksum0 = 12323221; + ObArrayHelper arr(1, &checksum0, 1); + uint8_t signature0 = 0; + ObArrayHelper sig(1, &signature0, 1); + uint64_t result = 0; + memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig); + EXPECT_EQ(result, 12323221); + EXPECT_EQ(0, sig.count()); + } + // multiple, but only one is valid + { + uint64_t checksum[64] = {12323221}; + for (int i = 1; i < 64; i++) checksum[i] = 1; + ObArrayHelper arr(64, checksum, 64); + uint8_t signature[64]; + ObArrayHelper sig(64, signature, 0); + uint64_t result = 0; + memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig); + EXPECT_EQ(result, 12323221); + EXPECT_EQ(0, sig.count()); + // valid is in middle + checksum[0] = 1; + checksum[13] = 34443; + memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig); + EXPECT_EQ(result, 34443); + EXPECT_EQ(0, sig.count()); + } + // multiple, multiple valid: 1, 13 + { + uint64_t checksum[64] = {12323221}; + for (int i = 1; i < 64; i++) checksum[i] = 1; + checksum[13] = 34443; + ObArrayHelper arr(64, checksum, 64); + uint8_t signature[64]; + ObArrayHelper sig(64, signature, 0); + uint64_t result = 0; + memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig); + EXPECT_GT(result, 1); + EXPECT_NE(result, 12323221); + EXPECT_NE(result, 34443); + EXPECT_EQ(64, sig.count()); + EXPECT_EQ(12323221 & 0xFF, sig.at(0)); + EXPECT_EQ(34443 & 0xFF, sig.at(13)); + } + // multiple, multiple valid, 18,21 + { + uint64_t checksum[64]; + for (int i = 0; i < 64; i++) checksum[i] = 1; + checksum[18] = 34443; + checksum[21] = 34444; + ObArrayHelper arr(64, checksum, 64); + uint8_t signature[64]; + ObArrayHelper sig(64, signature, 0); + uint64_t result = 0; + memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig); + EXPECT_GT(result, 1); + EXPECT_NE(result, 34443); + EXPECT_NE(result, 34444); + EXPECT_EQ(64, sig.count()); + EXPECT_EQ(34443 & 0xFF, sig.at(18)); + EXPECT_EQ(34444 & 0xFF, sig.at(21)); + } + // multiple, all is valid + { + uint64_t checksum[64] = {12323221}; + for (int i = 1; i < 64; i++) checksum[i] = 1 + i; + ObArrayHelper arr(64, checksum, 64); + uint8_t signature[64]; + ObArrayHelper sig(64, signature, 0); + uint64_t result = 0; + memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig); + EXPECT_NE(result, 12323221); + EXPECT_GT(result, 1); + EXPECT_EQ(64, sig.count()); + EXPECT_EQ(12323221 & 0xFF, sig.at(0)); + } +} + +}//end of unittest +}//end of oceanbase + +using namespace oceanbase; +using namespace oceanbase::common; + +int main(int argc, char **argv) +{ + int ret = 1; + ObLogger &logger = ObLogger::get_logger(); + logger.set_file_name("test_ob_tx_misc.log", true); + logger.set_log_level(OB_LOG_LEVEL_INFO); + testing::InitGoogleTest(&argc, argv); + ret = RUN_ALL_TESTS(); + return ret; +} diff --git a/unittest/storage/tx/test_ob_tx_log.cpp b/unittest/storage/tx/test_ob_tx_log.cpp index f9477fec56..cfc9f1b544 100644 --- a/unittest/storage/tx/test_ob_tx_log.cpp +++ b/unittest/storage/tx/test_ob_tx_log.cpp @@ -15,7 +15,8 @@ #define private public #include "storage/tx/ob_tx_log.h" #include "logservice/ob_log_base_header.h" - +#include "lib/container/ob_array_helper.h" +void ob_abort (void) __THROW {} namespace oceanbase { using namespace common; @@ -35,8 +36,7 @@ public: //const TEST TxID TEST_TX_ID = 1024; -int64_t TEST_CLUSTER_VERSION = 1; -int64_t TEST_LOG_NO = 1; +int64_t TEST_CLUSTER_VERSION = DATA_VERSION_4_3_0_0; ObAddr TEST_ADDR(ObAddr::VER::IPV4,"1.0.0.1",606); int TEST_TRANS_TYPE = 1; int TEST_SESSION_ID = 56831; @@ -54,10 +54,12 @@ common::ObString TEST_TRCE_INFO("trace_info_test"); LogOffSet TEST_LOG_OFFSET(10); int64_t TEST_COMMIT_VERSION = 190878; int64_t TEST_CHECKSUM = 29890209; +ObArray TEST_CHECKSUM_SIGNATURE_ARRAY; int64_t TEST_SCHEMA_VERSION = 372837; int64_t TEST_TX_EXPIRED_TIME = 12099087; int64_t TEST_LOG_ENTRY_NO = 1233; -auto TEST_MAX_SUBMITTED_SEQ_NO = ObTxSEQ(12345, 0); +ObTxSEQ TEST_MAX_SUBMITTED_SEQ_NO = ObTxSEQ(12345, 0); +ObTxSEQ TEST_SERIAL_FINAL_SEQ_NO = ObTxSEQ(12346, 0); LSKey TEST_LS_KEY; ObXATransID TEST_XID; @@ -108,39 +110,54 @@ OB_TX_SERIALIZE_MEMBER(NewTestLog, compat_bytes_, tx_id_1, tx_id_2, tx_id_3); TEST_F(TestObTxLog, tx_log_block_header) { TRANS_LOG(INFO, "called", "func", test_info_->name()); - TxID id = 0; int64_t pos = 0; ObTxLogBlock fill_block, replay_block; - ObTxLogBlockHeader fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); - ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header)); - + ObTxLogBlockHeader &fill_block_header = fill_block.get_header(); + fill_block_header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); + fill_block_header.set_serial_final(); + ASSERT_TRUE(fill_block_header.is_serial_final()); + ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill()); + fill_block.seal(TEST_TX_ID); // check log_block_header char *buf = fill_block.get_buf(); logservice::ObLogBaseHeader base_header_1; - logservice::ObLogBaseHeader base_header_2; + pos = 0; base_header_1.deserialize(buf, base_header_1.get_serialize_size(), pos); EXPECT_EQ(base_header_1.get_log_type() , ObTxLogBlock::DEFAULT_LOG_BLOCK_TYPE); EXPECT_EQ(base_header_1.get_replay_hint(), TEST_TX_ID); - ObTxLogBlockHeader replay_block_header; - ASSERT_EQ(OB_SUCCESS, - replay_block.init_with_header(buf, - fill_block.get_size(), - id, - replay_block_header)); + ObTxLogBlockHeader &replay_block_header = replay_block.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(buf, fill_block.get_size())); uint64_t tmp_cluster_id = replay_block_header.get_org_cluster_id(); EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id); - EXPECT_EQ(id, TEST_TX_ID); + EXPECT_EQ(replay_block.get_log_base_header().get_replay_hint(), TEST_TX_ID); + EXPECT_EQ(TEST_CLUSTER_VERSION, replay_block_header.get_cluster_version()); + EXPECT_EQ(TEST_LOG_ENTRY_NO, replay_block_header.get_log_entry_no()); + EXPECT_EQ(fill_block_header.flags(), replay_block_header.flags()); + EXPECT_TRUE(replay_block_header.is_serial_final()); - fill_block.reuse(id, replay_block_header); + // reuse + fill_block.get_header().init(TEST_ORG_CLUSTER_ID + 1, TEST_CLUSTER_VERSION + 1, TEST_LOG_ENTRY_NO + 1, ObTransID(TEST_TX_ID + 1), TEST_ADDR); + fill_block.reuse_for_fill(); + fill_block.seal(TEST_TX_ID + 1); buf = fill_block.get_buf(); pos = 0; + + logservice::ObLogBaseHeader base_header_2; base_header_2.deserialize(buf, base_header_2.get_serialize_size(), pos); EXPECT_EQ(base_header_2.get_log_type() , ObTxLogBlock::DEFAULT_LOG_BLOCK_TYPE); - EXPECT_EQ(base_header_2.get_replay_hint(), TEST_TX_ID); + EXPECT_EQ(base_header_2.get_replay_hint(), TEST_TX_ID + 1); + ObTxLogBlock replay_block2; + ObTxLogBlockHeader &replay_block_header2 = replay_block2.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block2.init_for_replay(buf, fill_block.get_size(), pos)); + EXPECT_EQ(TEST_ORG_CLUSTER_ID + 1, replay_block_header2.get_org_cluster_id()); + EXPECT_EQ(TEST_CLUSTER_VERSION + 1, replay_block_header2.get_cluster_version()); + EXPECT_EQ(TEST_LOG_ENTRY_NO + 1, replay_block_header2.get_log_entry_no()); + EXPECT_EQ(fill_block_header.flags(), replay_block_header2.flags()); + EXPECT_FALSE(replay_block_header2.is_serial_final()); } TEST_F(TestObTxLog, tx_log_body_except_redo) @@ -196,10 +213,12 @@ TEST_F(TestObTxLog, tx_log_body_except_redo) TEST_LAST_SCN, TEST_MAX_SUBMITTED_SEQ_NO, TEST_CLUSTER_VERSION, - TEST_XID); + TEST_XID, + TEST_SERIAL_FINAL_SEQ_NO); ObTxPrepareLog filll_prepare(TEST_LS_ARRAY, TEST_LOG_OFFSET); ObTxCommitLog fill_commit(share::SCN::base_scn(), TEST_CHECKSUM, + TEST_CHECKSUM_SIGNATURE_ARRAY, TEST_LS_ARRAY, TEST_TX_BUFFER_NODE_ARRAY, TEST_TRANS_TYPE, @@ -209,8 +228,9 @@ TEST_F(TestObTxLog, tx_log_body_except_redo) ObTxAbortLog fill_abort(TEST_TX_BUFFER_NODE_ARRAY); ObTxRecordLog fill_record(TEST_LOG_OFFSET, TEST_LOG_OFFSET_ARRY); - ObTxLogBlockHeader header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); - ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, header)); + ObTxLogBlockHeader &header = fill_block.get_header(); + header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); + ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill()); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_active_state)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit_state)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(filll_prepare)); @@ -218,13 +238,11 @@ TEST_F(TestObTxLog, tx_log_body_except_redo) ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_clear)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_abort)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_record)); - - TxID id = 0; + fill_block.seal(TEST_TX_ID); ObTxLogHeader tx_log_header; - ObTxLogBlockHeader block_header; - ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(fill_block.get_buf(), fill_block.get_size(), id, block_header)); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size())); - uint64_t tmp_cluster_id = block_header.get_org_cluster_id(); + uint64_t tmp_cluster_id = replay_block.get_header().get_org_cluster_id(); EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id); ObTxActiveInfoLogTempRef active_temp_ref; @@ -247,7 +265,9 @@ TEST_F(TestObTxLog, tx_log_body_except_redo) EXPECT_EQ(TEST_LAST_OP_SN, replay_active_state.get_last_op_sn()); EXPECT_EQ(TEST_FIRST_SCN, replay_active_state.get_first_seq_no()); EXPECT_EQ(TEST_LAST_SCN, replay_active_state.get_last_seq_no()); - EXPECT_EQ(TEST_CLUSTER_VERSION, replay_active_state.get_cluster_version()); + EXPECT_EQ(0, replay_active_state.get_cluster_version()); + EXPECT_EQ(TEST_XID, replay_active_state.get_xid()); + EXPECT_EQ(TEST_SERIAL_FINAL_SEQ_NO, replay_active_state.get_serial_final_seq_no()); ObTxCommitInfoLogTempRef commit_state_temp_ref; ObTxCommitInfoLog replay_commit_state(commit_state_temp_ref); @@ -322,28 +342,29 @@ TEST_F(TestObTxLog, tx_log_body_redo) TEST_EPOCH); ObTxCommitLog fill_commit(share::SCN::base_scn(), TEST_CHECKSUM, + TEST_CHECKSUM_SIGNATURE_ARRAY, TEST_LS_ARRAY, TEST_TX_BUFFER_NODE_ARRAY, TEST_TRANS_TYPE, TEST_LOG_OFFSET, TEST_INFO_ARRAY); - ObTxLogBlockHeader fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); - ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header)); + ObTxLogBlockHeader &fill_block_header = fill_block.get_header(); + fill_block_header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); + ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill()); ObString TEST_MUTATOR_BUF("FFF"); int64_t mutator_pos = 0; - ObTxRedoLog fill_redo(TEST_LOG_NO, TEST_CLUSTER_VERSION); + ObTxRedoLog fill_redo(TEST_CLUSTER_VERSION); ASSERT_EQ(OB_SUCCESS, fill_block.prepare_mutator_buf(fill_redo)); - ASSERT_EQ(OB_SUCCESS, - serialization::encode(fill_redo.get_mutator_buf(), - fill_redo.get_mutator_size(), - mutator_pos, - TEST_MUTATOR_BUF)); + ASSERT_EQ(OB_SUCCESS, serialization::encode(fill_redo.get_mutator_buf(), + fill_redo.get_mutator_size(), + mutator_pos, + TEST_MUTATOR_BUF)); ASSERT_EQ(OB_SUCCESS, fill_block.finish_mutator_buf(fill_redo, mutator_pos)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit_state)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit)); - + fill_block.seal(TEST_TX_ID); mutator_pos = 0; TxID id = 0; ObTxLogHeader log_header; @@ -351,11 +372,12 @@ TEST_F(TestObTxLog, tx_log_body_redo) ObTxRedoLogTempRef redo_temp_ref; ObTxRedoLog replay_redo(redo_temp_ref); - ObTxLogBlockHeader replay_block_header; - ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(fill_block.get_buf(), fill_block.get_size(), id, replay_block_header)); + ObTxLogBlockHeader &replay_block_header = replay_block.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size())); uint64_t tmp_cluster_id = replay_block_header.get_org_cluster_id(); EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id); + EXPECT_EQ(TEST_CLUSTER_VERSION, replay_block_header.get_cluster_version()); ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header)); EXPECT_EQ(ObTxLogType::TX_REDO_LOG, log_header.get_tx_log_type()); @@ -366,15 +388,13 @@ TEST_F(TestObTxLog, tx_log_body_redo) K(fill_redo.get_mutator_buf()), K(replay_redo.get_replay_mutator_buf()), K(replay_redo.get_mutator_size())); - ASSERT_EQ(OB_SUCCESS, - serialization::decode(replay_redo.get_replay_mutator_buf(), - replay_redo.get_mutator_size(), - mutator_pos, - replay_mutator_buf)); + ASSERT_EQ(OB_SUCCESS, serialization::decode(replay_redo.get_replay_mutator_buf(), + replay_redo.get_mutator_size(), + mutator_pos, + replay_mutator_buf)); EXPECT_EQ(TEST_MUTATOR_BUF, replay_mutator_buf); // EXPECT_EQ(TEST_CLOG_ENCRYPT_INFO,replay_redo.get_clog_encrypt_info()); - // EXPECT_EQ(TEST_LOG_NO,replay_redo.get_log_no()); - EXPECT_EQ(TEST_CLUSTER_VERSION,replay_redo.get_cluster_version()); + EXPECT_EQ(replay_redo.get_cluster_version(), 0); ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header)); EXPECT_EQ(ObTxLogType::TX_COMMIT_INFO_LOG, log_header.get_tx_log_type()); ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header)); @@ -382,8 +402,8 @@ TEST_F(TestObTxLog, tx_log_body_redo) //ignore replay log, only need commit log - ObTxLogBlockHeader replay_block_header_2; - ASSERT_EQ(OB_SUCCESS, replay_block_2.init_with_header(fill_block.get_buf(), fill_block.get_size(), id, replay_block_header_2)); + ObTxLogBlockHeader &replay_block_header_2 = replay_block_2.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block_2.init_for_replay(fill_block.get_buf(), fill_block.get_size())); tmp_cluster_id = replay_block_header_2.get_org_cluster_id(); EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id); @@ -398,7 +418,6 @@ TEST_F(TestObTxLog, tx_log_body_redo) // replay_mutator_buf)); // EXPECT_EQ(TEST_MUTATOR_BUF, replay_mutator_buf); // EXPECT_EQ(TEST_CLOG_ENCRYPT_INFO,replay_redo.get_clog_encrypt_info()); - // EXPECT_EQ(TEST_LOG_NO,replay_redo.get_log_no()); // EXPECT_EQ(TEST_CLUSTER_VERSION,replay_redo.get_cluster_version()); ASSERT_EQ(OB_SUCCESS, replay_block_2.get_next_log(log_header)); EXPECT_EQ(ObTxLogType::TX_COMMIT_INFO_LOG, log_header.get_tx_log_type()); @@ -492,8 +511,9 @@ TEST_F(TestObTxLog, test_default_log_deserialize) // ObTxLogBlockHeader fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, // ObTransID(TEST_TX_ID)); - ObTxLogBlockHeader fill_block_header; - ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header)); + ObTxLogBlockHeader &fill_block_header = fill_block.get_header(); + fill_block_header.init(1, TEST_CLUSTER_VERSION, 2, ObTransID(3), TEST_ADDR); + ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill()); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_active_state)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit_state)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_prepare)); @@ -501,14 +521,14 @@ TEST_F(TestObTxLog, test_default_log_deserialize) ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_clear)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_abort)); ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_record)); + fill_block.seal(TEST_TX_ID); TxID id = 0; int64_t fill_member_cnt = 0; int64_t replay_member_cnt = 0; ObTxLogHeader tx_log_header; - ObTxLogBlockHeader replay_block_header; - ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(fill_block.get_buf(), fill_block.get_size(), - id, replay_block_header)); + ObTxLogBlockHeader &replay_block_header = replay_block.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size())); fill_member_cnt = fill_block_header.compat_bytes_.total_obj_cnt_; EXPECT_EQ(fill_block_header.get_org_cluster_id(), replay_block_header.get_org_cluster_id()); replay_member_cnt++; @@ -518,7 +538,11 @@ TEST_F(TestObTxLog, test_default_log_deserialize) replay_member_cnt++; EXPECT_EQ(fill_block_header.get_scheduler(), replay_block_header.get_scheduler()); replay_member_cnt++; - EXPECT_EQ(replay_member_cnt, fill_member_cnt); + EXPECT_EQ(fill_block_header.get_cluster_version(), replay_block_header.get_cluster_version()); + replay_member_cnt++; + EXPECT_EQ(fill_block_header.flags(), replay_block_header.flags()); + replay_member_cnt++; + EXPECT_EQ(replay_member_cnt, fill_member_cnt - 1/*1 skipped*/); ObTxActiveInfoLogTempRef active_temp_ref; ObTxActiveInfoLog replay_active_state(active_temp_ref); @@ -564,6 +588,8 @@ TEST_F(TestObTxLog, test_default_log_deserialize) replay_member_cnt++; EXPECT_EQ(fill_active_state.get_xid(), replay_active_state.get_xid()); replay_member_cnt++; + EXPECT_EQ(fill_active_state.get_serial_final_seq_no(), replay_active_state.get_serial_final_seq_no()); + replay_member_cnt++; EXPECT_EQ(replay_member_cnt, fill_member_cnt); ObTxCommitInfoLogTempRef commit_state_temp_ref; @@ -703,10 +729,11 @@ void test_big_commit_info_log(int64_t log_size) ObTxCommitInfoLog fill_commit_state(TEST_ADDR, TEST_LS_ARRAY, TEST_LS_KEY, TEST_IS_SUB2PC, TEST_IS_DUP, TEST_CAN_ELR, TEST_TRACE_ID_STR, TEST_TRCE_INFO, TEST_LOG_OFFSET, TEST_BIG_REDO_LSN_ARRAY, TEST_LS_ARRAY, - TEST_CLUSTER_VERSION, TEST_XID, TEST_COMMIT_PARTS, TEST_EPOCH); - ObTxLogBlockHeader - fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); - ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header)); + TEST_CLUSTER_VERSION, TEST_XID, + TEST_COMMIT_PARTS, TEST_EPOCH); + ObTxLogBlockHeader &fill_block_header = fill_block.get_header(); + fill_block_header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR); + ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill()); ASSERT_EQ(OB_LOG_TOO_LARGE, fill_block.add_new_log(fill_commit_state, &fill_big_segment)); const char *submit_buf = nullptr; @@ -716,19 +743,17 @@ void test_big_commit_info_log(int64_t log_size) int ret = OB_SUCCESS; while (OB_SUCC(ret) && OB_EAGAIN - == (ret = (fill_block.acquire_segment_log_buf(submit_buf, submit_buf_len, - fill_block_header, - ObTxLogType::TX_COMMIT_INFO_LOG)))) { + == (ret = (fill_block.acquire_segment_log_buf(ObTxLogType::TX_COMMIT_INFO_LOG)))) { share::SCN tmp_scn; EXPECT_EQ(OB_SUCCESS, tmp_scn.convert_for_inner_table_field(part_count)); if (OB_FAIL(fill_block.set_prev_big_segment_scn(tmp_scn))) { TRANS_LOG(WARN, "set prev big segment scn failed", K(ret), K(part_count)); + } else if (OB_FAIL(fill_block.seal(TEST_TX_ID))) { + TRANS_LOG(WARN, "seal block fail", K(ret)); } else { replay_block.reset(); - TxID id = 0; - ObTxLogBlockHeader replay_block_header; - ASSERT_EQ(OB_SUCCESS, - replay_block.init_with_header(submit_buf, submit_buf_len, id, replay_block_header)); + ObTxLogBlockHeader &replay_block_header = replay_block.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size())); if (OB_FAIL(replay_block.get_next_log(log_type_header, &replay_big_segment))) { TRANS_LOG(WARN, "get next log failed", K(ret), K(part_count)); EXPECT_EQ(OB_LOG_TOO_LARGE, ret); @@ -742,11 +767,10 @@ void test_big_commit_info_log(int64_t log_size) } // EXPECT_EQ(ObTxLogType::TX_COMMIT_INFO_LOG, log_type_header.get_tx_log_type()); if (OB_ITER_END == ret) { + fill_block.seal(TEST_TX_ID); replay_block.reset(); - TxID id = 0; - ObTxLogBlockHeader replay_block_header; - ASSERT_EQ(OB_SUCCESS, - replay_block.init_with_header(submit_buf, submit_buf_len, id, replay_block_header)); + ObTxLogBlockHeader &replay_block_header = replay_block.get_header(); + ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size())); if (OB_FAIL(replay_block.get_next_log(log_type_header, &replay_big_segment))) { TRANS_LOG(WARN, "get next log failed", K(ret), K(part_count)); } @@ -790,7 +814,7 @@ void test_big_commit_info_log(int64_t log_size) EXPECT_EQ(fill_commit_state.get_incremental_participants().count(), replay_commit_state.get_incremental_participants().count()); replay_member_cnt++; - EXPECT_EQ(fill_commit_state.get_cluster_version(), replay_commit_state.get_cluster_version()); + EXPECT_EQ(0, replay_commit_state.get_cluster_version()); replay_member_cnt++; EXPECT_EQ(fill_commit_state.get_app_trace_id().length(), replay_commit_state.get_app_trace_id().length()); @@ -818,8 +842,115 @@ TEST_F(TestObTxLog, test_big_segment_log) test_big_commit_info_log(10*1024*1024); } -} // namespace unittest +TEST_F(TestObTxLog, test_commit_log_with_checksum_signature) +{ + uint64_t checksum = 0; + uint8_t sig[64]; + ObArrayHelper checksum_signatures(64, sig); + for(int i = 0; i< 64; i++) { + uint64_t checksum_i = ObRandom::rand(1, 99999); + checksum = ob_crc64(checksum, &checksum_i, sizeof(checksum_i)); + checksum_signatures.push_back((uint8_t)(checksum_i & 0xFF)); + } + ObLSArray ls_array; + ls_array.push_back(ObLSID(1001)); + ObTxBufferNodeArray tx_buffer_node_array; + ObTxBufferNode node; + ObString str("hello,world"); + node.init(ObTxDataSourceType::LS_TABLE, str, share::SCN(), nullptr); + tx_buffer_node_array.push_back(node); + ObLSLogInfoArray ls_info_array; + ls_info_array.push_back(ObLSLogInfo()); + share::SCN scn; + scn.convert_for_tx(101010101010101); + ObTxCommitLog log0(scn, + checksum, + checksum_signatures, + ls_array, + tx_buffer_node_array, + 1, + LogOffSet(100), + ls_info_array); + int64_t size = log0.get_serialize_size(); + char *buf = new char[size]; + int64_t pos = 0; + ASSERT_EQ(OB_SUCCESS, log0.serialize(buf, size, pos)); + ObTxCommitLogTempRef ref; + ObTxCommitLog log1(ref); + pos = 0; + ASSERT_EQ(OB_SUCCESS, log1.deserialize(buf, size, pos)); + ASSERT_EQ(log1.checksum_, log0.checksum_); + ASSERT_EQ(log1.checksum_, checksum); + ASSERT_EQ(log1.checksum_sig_.count(), 64); + for(int i = 0; i < log1.checksum_sig_.count(); i++) { + ASSERT_EQ(log1.checksum_sig_.at(i), sig[i]); + } +} +TEST_F(TestObTxLog, test_start_working_log) +{ + ObTransID fake_tx_id(0); + ObTxLogBlockHeader header(1, 1, 1, fake_tx_id, ObAddr()); + EXPECT_EQ(0, header.get_serialize_size_()); + EXPECT_EQ(OB_SUCCESS, header.before_serialize()); + int64_t ser_size_ = header.get_serialize_size_(); + int64_t ser_size = header.get_serialize_size(); + EXPECT_NE(0, ser_size_); + char buf[256]; + MEMSET(buf, 0, 256); + int64_t pos = 0; + EXPECT_EQ(OB_SUCCESS, header.serialize(buf, 256, pos)); + EXPECT_EQ(pos, ser_size); + ObTxLogBlockHeader header2; + int64_t pos0 = 0; + EXPECT_EQ(OB_SUCCESS, header2.deserialize(buf, 256, pos0)); + EXPECT_LE(pos0, pos); + EXPECT_EQ(header2.tx_id_, fake_tx_id); + EXPECT_EQ(header2.cluster_version_, header.cluster_version_); + EXPECT_EQ(header2.log_entry_no_, header.log_entry_no_); +} + +TEST_F(TestObTxLog, test_tx_block_header_serialize) +{ + // 1. user must call before_serialize, before get_serialize_size(), serialize() + ObTransID tx_id(1024); + ObAddr addr(ObAddr::VER::IPV4, "127.2.3.4", 2048); + ObTxLogBlockHeader header(101, 102, 103, tx_id, addr); + EXPECT_EQ(0, header.serialize_size_); + EXPECT_EQ(OB_SUCCESS, header.before_serialize()); + int64_t ser_size_ = header.get_serialize_size_(); + EXPECT_EQ(ser_size_, header.get_serialize_size_()); + int64_t ser_size = header.get_serialize_size(); + EXPECT_GT(ser_size_, 0); + char buf[256]; + MEMSET(buf, 0, 256); + int64_t pos = 0; + EXPECT_EQ(OB_SUCCESS, header.serialize(buf, 256, pos)); + EXPECT_EQ(pos, ser_size); + + // test deserialize ok + ObTxLogBlockHeader header2; + int64_t pos0 = 0; + EXPECT_EQ(OB_SUCCESS, header2.deserialize(buf, 256, pos0)); + EXPECT_LE(pos0, pos); + EXPECT_EQ(header2.tx_id_, tx_id); + EXPECT_EQ(header2.org_cluster_id_, 101); + EXPECT_EQ(header2.cluster_version_, 102); + EXPECT_EQ(header2.log_entry_no_, 103); + EXPECT_EQ(header2.scheduler_, addr); + + // the serilize size is always equals to header.serialize_size_ + header.serialize_size_ = 240; + int64_t ser_size2 = header.get_serialize_size(); + EXPECT_GT(ser_size2, 240); + EXPECT_EQ(240, header.get_serialize_size_()); + MEMSET(buf, 0, 256); + pos = 0; + EXPECT_EQ(OB_SUCCESS, header.serialize(buf, 256, pos)); + EXPECT_EQ(pos, ser_size2); +} + +} // namespace unittest } // namespace oceanbase using namespace oceanbase; diff --git a/unittest/storage/tx/test_ob_tx_msg.cpp b/unittest/storage/tx/test_ob_tx_msg.cpp index 79879386af..66187b80ce 100644 --- a/unittest/storage/tx/test_ob_tx_msg.cpp +++ b/unittest/storage/tx/test_ob_tx_msg.cpp @@ -121,7 +121,7 @@ public: msg.request_id_ = op_sn_; msg.savepoint_ = ObTxSEQ(1, 0); msg.op_sn_ = op_sn_; - msg.branch_id_ = 1; + msg.tx_seq_base_ = 10000000001; msg.tx_ptr_ = tx; } void build_tx_keepalive_msg(ObTxKeepaliveMsg &msg) @@ -432,7 +432,7 @@ TEST_F(TestObTxMsg, trans_rollback_sp_msg) EXPECT_EQ(msg.cluster_id_, msg1.cluster_id_); EXPECT_EQ(msg.savepoint_, msg1.savepoint_); EXPECT_EQ(msg.op_sn_, msg1.op_sn_); - EXPECT_EQ(msg.branch_id_, msg1.branch_id_); + EXPECT_EQ(msg.tx_seq_base_, msg1.tx_seq_base_); EXPECT_EQ(msg.tx_ptr_->parts_[0].id_, msg1.tx_ptr_->parts_[0].id_); if (OB_NOT_NULL(msg.tx_ptr_)) { msg.tx_ptr_ = NULL; diff --git a/unittest/storage/tx/test_redo_submitter.cpp b/unittest/storage/tx/test_redo_submitter.cpp new file mode 100644 index 0000000000..dc20fad0e1 --- /dev/null +++ b/unittest/storage/tx/test_redo_submitter.cpp @@ -0,0 +1,650 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include +#include +#include +#define private public +#define protected public +#include "storage/tx/ob_trans_define.h" +#include "storage/tx/ob_trans_service.h" +#include "storage/tx/ob_trans_part_ctx.h" +#include "storage/tx/ob_tx_redo_submitter.h" +#define USING_LOG_PREFIX TRANS + + +namespace oceanbase +{ +using namespace ::testing; +using namespace transaction; +using namespace share; +using namespace memtable; +namespace transaction { +// Test submitter logic works fine in use cases +// +// mock part_ctx's interface: +// - prepare_for_submit_redo +// - fill_log_block +// - submit_log_block_out +// - is_parallel_logging +// +// mock memtable_ctx's interface +// - fill_redo_log +// - log_submitted +// +struct MockDelegate { + virtual bool is_parallel_logging() const = 0; + virtual int submit_redo_log_out(ObTxLogBlock &log_block, + ObTxLogCb *&log_cb, + memtable::ObRedoLogSubmitHelper &helper, + const int64_t replay_hint, + const bool has_hold_ctx_lock, + share::SCN &submitted) = 0; + virtual int fill_redo_log(memtable::ObTxFillRedoCtx &ctx) = 0; +}; +struct MockImpl : MockDelegate { + int a_; + public: + MOCK_CONST_METHOD0(is_parallel_logging, bool()); + MOCK_METHOD6(submit_redo_log_out, int(ObTxLogBlock &, + ObTxLogCb *&, + memtable::ObRedoLogSubmitHelper &, + const int64_t, + const bool, + share::SCN &)); + MOCK_METHOD1(fill_redo_log, int(memtable::ObTxFillRedoCtx &)); +}; +thread_local MockImpl *mock_ptr; +class ObTestRedoSubmitter : public ::testing::Test +{ +public: + virtual void SetUp() override + { + oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION); + ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001); + ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1",2023); + ObCurTraceId::init(ip_port); + GCONF._ob_trans_rpc_timeout = 500; + ObClockGenerator::init(); + const testing::TestInfo* const test_info = + testing::UnitTest::GetInstance()->current_test_info(); + auto test_name = test_info->name(); + _TRANS_LOG(INFO, ">>>> starting test : %s", test_name); + + // prepare for test + tx_ctx.exec_info_.state_ = ObTxState::INIT; + tx_ctx.exec_info_.scheduler_ = common::ObAddr(common::ObAddr::VER::IPV4, "127.0.0.1", 8888); + tx_ctx.exec_info_.next_log_entry_no_ = 0; + tx_ctx.cluster_version_ = DATA_CURRENT_VERSION; + ObLSID ls_id(1001); ObTransID tx_id(777); + EXPECT_EQ(OB_SUCCESS,tx_ctx.init_log_cbs_(ls_id, tx_id)); + mock_ptr = &mdo_; + } + virtual void TearDown() override + { + const testing::TestInfo* const test_info = + testing::UnitTest::GetInstance()->current_test_info(); + auto test_name = test_info->name(); + _TRANS_LOG(INFO, ">>>> tearDown test : %s", test_name); + ObClockGenerator::destroy(); + ObMallocAllocator::get_instance()->recycle_tenant_allocator(1001); + } + MockImpl mdo_; + ObPartTransCtx tx_ctx; + ObMemtableCtx mt_ctx; +}; + +int succ_submit_redo_log_out(ObTxLogBlock & b, + ObTxLogCb *& log_cb, + memtable::ObRedoLogSubmitHelper &h, + const int64_t replay_hint, + const bool has_hold_ctx_lock, + share::SCN &submitted_scn) +{ + log_cb = NULL; + submitted_scn.convert_for_tx(123123123); + return OB_SUCCESS; +} +bool ObPartTransCtx::is_parallel_logging() const +{ + return mock_ptr->is_parallel_logging(); +} + +int ObPartTransCtx::submit_redo_log_out(ObTxLogBlock &log_block, + ObTxLogCb *&log_cb, + memtable::ObRedoLogSubmitHelper &helper, + const int64_t replay_hint, + const bool has_hold_ctx_lock, + share::SCN &submitted_scn) +{ + return mock_ptr->submit_redo_log_out(log_block, log_cb, helper, replay_hint, has_hold_ctx_lock, submitted_scn); +} +}// transaction + +namespace memtable { +int ObMemtableCtx::fill_redo_log(ObTxFillRedoCtx &ctx) +{ + TRANS_LOG(INFO, "", K(mock_ptr->a_)); + return mock_ptr->fill_redo_log(ctx); +} +int ObMemtableCtx::log_submitted(const memtable::ObRedoLogSubmitHelper &helper) +{ + TRANS_LOG(INFO, "", K(mock_ptr->a_)); + return OB_SUCCESS; +} +int ObMemtableCtx::get_log_guard(const transaction::ObTxSEQ &write_seq, + memtable::ObCallbackListLogGuard &log_guard, + int& cb_list_idx) +{ + TRANS_LOG(INFO, "", K(mock_ptr->a_)); + cb_list_idx = write_seq.get_branch(); + return OB_SUCCESS; +} +} // memtable +namespace transaction { +TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_BLOCK_FROZEN) +{ + mdo_.a_ = 1; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(false)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BLOCK_FROZEN; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_BLOCK_FROZEN, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_BUF_NOT_ENOUGH) +{ + mdo_.a_ = 2; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(false)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BUF_NOT_ENOUGH; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_ALL_FILLED) +{ + mdo_.a_ = 2; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(false)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_SUCCESS; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_UNEXPECTED_ERROR) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(false)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_ALLOCATE_MEMORY_FAILED; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_ALLOCATE_MEMORY_FAILED, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_serial_final) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(false)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_SUCCESS; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, true/*serial final*/)); + } +} + +TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_BLOCK_FROZEN) +{ + mdo_.a_ = 4; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BLOCK_FROZEN; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxSEQ writer_seq(101, 0); + EXPECT_EQ(OB_BLOCK_FROZEN, submitter.parallel_submit(writer_seq)); + } +} + +TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_BLOCKED_BY_OTHER_LIST) +{ + mdo_.a_ = 4; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_ITER_END; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxSEQ writer_seq(101, 0); + EXPECT_EQ(OB_ITER_END, submitter.parallel_submit(writer_seq)); + } +} + +TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_ALL_FILLED) +{ + mdo_.a_ = 4; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_SUCCESS; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxSEQ writer_seq(101, 0); + EXPECT_EQ(OB_SUCCESS, submitter.parallel_submit(writer_seq)); + } +} + +TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_UNEXPECTED_ERROR) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_ALLOCATE_MEMORY_FAILED; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxSEQ writer_seq(101, 0); + EXPECT_EQ(OB_ALLOCATE_MEMORY_FAILED, submitter.parallel_submit(writer_seq)); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_BLOCK_FROZEN) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BLOCK_FROZEN; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_BLOCK_FROZEN, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_BLOCKED_BY_OTHERS) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_ITER_END; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_ITER_END, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_CURRENT_FILLED_BUT_OTHERS_REMAINS) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_EAGAIN; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_EAGAIN, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_BUF_NOT_ENOUGH) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BUF_NOT_ENOUGH; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_ALL_FILLED) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_SUCCESS; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false)); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_switch_leader_or_on_commit_serial_logging) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(false)); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BLOCK_FROZEN; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(1) + .WillOnce(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxLogBlock log_block; + ObTransID tx_id(101); + log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr()); + log_block.init_for_fill(); + memtable::ObRedoLogSubmitHelper helper; + EXPECT_EQ(OB_BLOCK_FROZEN, submitter.fill(log_block, helper)); + EXPECT_EQ(0, helper.callbacks_.count()); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_switch_leader_or_on_commit_parallel_logging_ALL_FILLED) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(4) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BLOCK_FROZEN; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 200; + ctx.buf_pos_ = 333; + return OB_ITER_END; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 300; + ctx.buf_pos_ = 444; + return OB_EAGAIN; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 400; + ctx.buf_pos_ = 555; + ObCallbackScope scope; + ctx.helper_->callbacks_.push_back(scope); + return OB_SUCCESS; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(3) + .WillRepeatedly(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxLogBlock log_block; + ObTransID tx_id(101); + log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr()); + log_block.init_for_fill(); + memtable::ObRedoLogSubmitHelper helper; + EXPECT_EQ(OB_SUCCESS, submitter.fill(log_block, helper)); + EXPECT_EQ(helper.callbacks_.count(), 1); + } +} + +TEST_F(ObTestRedoSubmitter, submit_by_switch_leader_or_on_commit_parallel_logging_BLOCKED) +{ + mdo_.a_ = 3; + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(4) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 100; + ctx.buf_pos_ = 200; + return OB_BLOCK_FROZEN; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 200; + ctx.buf_pos_ = 333; + return OB_ITER_END; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 300; + ctx.buf_pos_ = 444; + return OB_EAGAIN; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 0; + ctx.buf_pos_ = 0; + return OB_BLOCK_FROZEN; + })); + EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_)) + .Times(3) + .WillRepeatedly(Invoke(succ_submit_redo_log_out)); + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxLogBlock log_block; + ObTransID tx_id(101); + log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr()); + log_block.init_for_fill(); + memtable::ObRedoLogSubmitHelper helper; + EXPECT_EQ(OB_BLOCK_FROZEN, submitter.fill(log_block, helper)); + EXPECT_EQ(helper.callbacks_.count(), 0); + } +} + +TEST_F(ObTestRedoSubmitter, submit_ROW_SIZE_TOO_LARGE) +{ + EXPECT_CALL(mdo_, is_parallel_logging()) + .Times(AtLeast(1)) + .WillRepeatedly(Return(true)); + { + EXPECT_CALL(mdo_, fill_redo_log(_)) + .Times(3) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx) { + ctx.fill_count_ = 0; + ctx.buf_pos_ = 0; + return OB_ERR_TOO_BIG_ROWSIZE; + })); + { + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + EXPECT_EQ(OB_ERR_TOO_BIG_ROWSIZE, submitter.submit(true, false, true)); + EXPECT_EQ(submitter.get_submitted_cnt(), 0); + } + { + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxSEQ write_seq(100,200); + EXPECT_EQ(OB_ERR_TOO_BIG_ROWSIZE, submitter.parallel_submit(write_seq)); + EXPECT_EQ(submitter.get_submitted_cnt(), 0); + } + { + ObTxRedoSubmitter submitter(tx_ctx, mt_ctx); + ObTxLogBlock log_block; + ObTransID tx_id(101); + log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr()); + log_block.init_for_fill(); + memtable::ObRedoLogSubmitHelper helper; + EXPECT_EQ(OB_ERR_TOO_BIG_ROWSIZE, submitter.fill(log_block, helper)); + EXPECT_EQ(submitter.get_submitted_cnt(), 0); + EXPECT_EQ(helper.callbacks_.count(), 0); + } + } +} + +} // transaction +} // oceanbase + +int main(int argc, char **argv) +{ + const char *log_name = "test_redo_submitter.log"; + system("rm -rf test_redo_submitter.log*"); + ObLogger &logger = ObLogger::get_logger(); + logger.set_file_name(log_name, true, false, + log_name, + log_name, + log_name); + logger.set_log_level(OB_LOG_LEVEL_DEBUG); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/unittest/storage/tx/test_trans_callback_mgr_fill_redo.cpp b/unittest/storage/tx/test_trans_callback_mgr_fill_redo.cpp new file mode 100644 index 0000000000..beeb3f72ba --- /dev/null +++ b/unittest/storage/tx/test_trans_callback_mgr_fill_redo.cpp @@ -0,0 +1,774 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include +#include +#include +#define private public +#define protected public +#include "storage/tx/ob_trans_define.h" +#include "storage/tx/ob_trans_service.h" +#include "storage/tx/ob_trans_part_ctx.h" +#include "storage/tx/ob_tx_redo_submitter.h" +#define USING_LOG_PREFIX TRANS + +namespace oceanbase +{ +using namespace ::testing; +using namespace transaction; +using namespace share; +namespace memtable { +// Test fill redo logic works for multi-callback-list +// +// target interface to test: +// - fill_from_one_list +// - fill_from_all_list +// +// target situation: +// - BLOCK_FROZEN +// - blocked by other list +// - all callback filled successfully +// +// logging mode: +// - parallel logging +// - serial logging +// +// mock callback list interface +// - fill_log +// - get_log_epoch +// +struct MockDelegate { + virtual int fill_log(ObTxFillRedoCtx &fill_ctx) = 0; + virtual int64_t get_log_epoch(int i) const = 0; + virtual int get_logging_list_count() const = 0; +}; +struct MockImpl : MockDelegate { + struct MockCallback : public ObITransCallback + { + }; + int a_; + ObTxFillRedoCtx fill_ctx_; + MockCallback cb1_; + MockCallback cb2_; + MockCallback cb3_; + MockImpl() : cb1_() { + cb1_.next_ = &cb2_; + cb1_.prev_ = &cb3_; + cb2_.next_ = &cb3_; + cb2_.prev_ = &cb1_; + cb3_.next_ = &cb1_; + cb3_.prev_ = &cb2_; + } + void init_callback_scope() + { + fill_ctx_.callback_scope_->start_.cur_ = &cb1_; + fill_ctx_.callback_scope_->end_.cur_ = &cb2_; + } + public: + MOCK_CONST_METHOD0(get_logging_list_count, int()); + MOCK_CONST_METHOD1(get_log_epoch, int64_t(int)); + MOCK_METHOD1(fill_log, int(ObTxFillRedoCtx&)); +}; +thread_local MockImpl *mock_ptr; +int ObTxCallbackList::fill_log(ObITransCallback* cursor, ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &func) +{ + mock_ptr->init_callback_scope(); + return mock_ptr->fill_log(ctx); +} +int64_t ObTxCallbackList::get_log_epoch() const +{ + return mock_ptr->get_log_epoch(id_); +} +int ObTransCallbackMgr::get_logging_list_count() const +{ + return mock_ptr->get_logging_list_count(); +} +class ObTestRedoFill : public ::testing::Test +{ +public: + ObTestRedoFill(): + mem_ctx_(), + cb_allocator_(), + callback_mgr_(mem_ctx_, cb_allocator_) {} + virtual void SetUp() override + { + oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION); + ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001); + ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1",2023); + ObCurTraceId::init(ip_port); + GCONF._ob_trans_rpc_timeout = 500; + ObClockGenerator::init(); + const testing::TestInfo* const test_info = + testing::UnitTest::GetInstance()->current_test_info(); + auto test_name = test_info->name(); + _TRANS_LOG(INFO, ">>>> starting test : %s", test_name); + mdo_.fill_ctx_.helper_ = &helper_; + mock_ptr = &mdo_; + cb_allocator_.init(1001); + } + virtual void TearDown() override + { + const testing::TestInfo* const test_info = + testing::UnitTest::GetInstance()->current_test_info(); + auto test_name = test_info->name(); + _TRANS_LOG(INFO, ">>>> tearDown test : %s", test_name); + ObClockGenerator::destroy(); + ObMallocAllocator::get_instance()->recycle_tenant_allocator(1001); + } + void extend_callback_lists_(int cnt) + { + callback_mgr_.callback_lists_ = (ObTxCallbackList*)new char[(sizeof(ObTxCallbackList) * cnt)]; + for (int i=0; i {100,100,98,101} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 2: 97 -> 100 + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 97; + ctx.next_epoch_ = 100; + return OB_ITER_END; + })); + ++i; + // -> {100,100,101,101} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 3: 98 -> 101 + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 98; + ctx.next_epoch_ = 101; + return OB_ITER_END; + })); + ++i; + // -> {102,100,101,101} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 1: 100 -> 102 + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 100; + ctx.next_epoch_ = 102; + return OB_ITER_END; + })); + ++i; + // -> {102,102,101,101} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 2: 101 -> 102 + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 100; + ctx.next_epoch_ = 102; + return OB_ITER_END; + })); + ++i; + // -> {102,102,END,101} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 3: 101 -> END + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 101; + return OB_SUCCESS; + })); + ++i; + // -> {102,102,END,END} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 4: 101 -> END + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 101; + return OB_SUCCESS; + })); + ++i; + // -> {END,102,END,END} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 1: 102 -> END + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 102; + return OB_SUCCESS; + })); + ++i; + // -> {END,END,END,END} + EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + // list 2: 102 -> END + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 102; + return OB_SUCCESS; + })); + ++i; + } + auto &ctx = mdo_.fill_ctx_; + EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(ctx, fill_func)); + EXPECT_EQ(ctx.epoch_from_, 102); + EXPECT_EQ(ctx.epoch_to_, 102); + EXPECT_EQ(ctx.fill_count_, i); + EXPECT_EQ(ctx.list_log_epoch_arr_[0], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[1], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[2], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[3], INT64_MAX); +} +TEST_F(ObTestRedoFill, serial_multi_list_fill_all_BLOCK_FROZEN) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100, 100, 100, 100}; + return epochs[i]; + })); + { + InSequence s; + // list 1,2,3 blocked + EXPECT_CALL(mdo_, fill_log(_)).Times(3) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 100; + return OB_BLOCK_FROZEN; + })); + // list 4 filled + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 10; + ctx.cur_epoch_ = 100; + return OB_SUCCESS; + })); + // next round + // fill list 1,2,3 again success + EXPECT_CALL(mdo_, fill_log(_)).Times(3) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 100; + return OB_SUCCESS; + })); + } + auto &ctx = mdo_.fill_ctx_; + EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(ctx, fill_func)); + EXPECT_EQ(ctx.epoch_from_, 100); + EXPECT_EQ(ctx.epoch_to_, 100); + EXPECT_EQ(ctx.fill_count_, 313); + EXPECT_EQ(ctx.list_log_epoch_arr_[0], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[1], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[2], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[3], INT64_MAX); +} + +TEST_F(ObTestRedoFill, serial_single_list_fill_all_BUF_NOT_ENOUGH) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // single list + callback_mgr_.callback_lists_ = NULL; + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(1)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 999; + return OB_BUF_NOT_ENOUGH; + })); + EXPECT_EQ(OB_BUF_NOT_ENOUGH, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 999); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 999); +} + +TEST_F(ObTestRedoFill, serial_single_list_fill_all_list_BIG_ROW) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // single list + callback_mgr_.callback_lists_ = NULL; + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(1)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 0; + ctx.cur_epoch_ = 100; + return OB_BUF_NOT_ENOUGH; + })); + EXPECT_EQ(OB_BUF_NOT_ENOUGH, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 100); +} + +TEST_F(ObTestRedoFill, serial_multi_list_fill_all_list_BIG_ROW) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,99,100,99}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 0; + ctx.cur_epoch_ = 99; + return OB_BUF_NOT_ENOUGH; + })); + EXPECT_EQ(OB_BUF_NOT_ENOUGH, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 99); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 1); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 100); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 99); +} + +TEST_F(ObTestRedoFill, serial_multi_list_fill_all_OTHER_ERROR_WHEN_FILL_OTHERS) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100, 100, 100, 100}; + return epochs[i]; + })); + { + InSequence s; + // list 1 ALL FILLED + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 100; + return OB_SUCCESS; + })); + // list 2 FROZEN BLOCKED + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 10; + ctx.cur_epoch_ = 100; + return OB_BLOCK_FROZEN; + })); + // list 3 MEMORY_ALLOCATE_FAILED + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 100; + return OB_ALLOCATE_MEMORY_FAILED; + })); + } + auto &ctx = mdo_.fill_ctx_; + EXPECT_EQ(OB_ALLOCATE_MEMORY_FAILED, callback_mgr_.fill_from_all_list(ctx, fill_func)); + EXPECT_EQ(ctx.epoch_from_, 100); + EXPECT_EQ(ctx.epoch_to_, 100); + EXPECT_EQ(ctx.fill_count_, 111); + EXPECT_EQ(ctx.list_log_epoch_arr_[0], INT64_MAX); + EXPECT_EQ(ctx.list_log_epoch_arr_[1], 100); + EXPECT_EQ(ctx.list_log_epoch_arr_[2], 100); + EXPECT_EQ(ctx.list_log_epoch_arr_[3], 100); +} + +TEST_F(ObTestRedoFill, serial_logging_fill_from_all_list_ALL_OTHERS_BLOCK_FROZEN_EMPTY) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,100,100,100}; + return epochs[i]; + })); + { + InSequence s; + // list 1 ALL FILLED + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 100; + return OB_SUCCESS; + })); + // list 2,3,4 BLOCK_FROZEN AND NOTHING FILLED + EXPECT_CALL(mdo_, fill_log(_)).Times(3) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 0; + ctx.cur_epoch_ = 100; + return OB_BLOCK_FROZEN; + })); + } + EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 100); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 1); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 3); // point to list 4 + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], INT64_MAX); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 100); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 100); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[3], 100); +} + +TEST_F(ObTestRedoFill, serial_logging_fill_from_all_list_FIRST_ITER_END_OTHERS_BLOCK_FROZEN_EMPTY) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,101,100,200}; + return epochs[i]; + })); + { + InSequence s; + // list 1 ITER END + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 1; + ctx.cur_epoch_ = 100; + ctx.next_epoch_ = 101; + return OB_ITER_END; + })); + // list 2 skipped + // list 3, BLOCK_FROZEN AND NOTHING FILLED + EXPECT_CALL(mdo_, fill_log(_)).Times(1) + .WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 0; + ctx.cur_epoch_ = 100; + return OB_BLOCK_FROZEN; + })); + // list 4 skipped + } + EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 100); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 1); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); // point to list 3 + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 101); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 101); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 100); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[3], 200); +} + +TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_FROZEN) +{ + set_parallel_logging(true); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,99,98,99}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 99; + return OB_BLOCK_FROZEN; + })); + EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99); + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 99); +} + +TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_FROZEN_EMPTY) +{ + set_parallel_logging(true); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,99,98,99}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 0; + ctx.cur_epoch_ = 98; + return OB_BLOCK_FROZEN; + })); + EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 98); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 0); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 98); +} + +TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_FROZEN_FILL_FROM_OTHERS) +{ + set_parallel_logging(true); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,98,98,99}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(2) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 0; + ctx.cur_epoch_ = 98; + return OB_BLOCK_FROZEN; + })) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 99; + ctx.next_epoch_ = 101; + return OB_ITER_END; + })); + EXPECT_EQ(OB_ITER_END, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 98); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 98); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 101); +} + +TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_BY_OTHERS) +{ + set_parallel_logging(true); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,99,98,99}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 98; + ctx.next_epoch_ = 101; + return OB_ITER_END; + })); + EXPECT_EQ(OB_ITER_END, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99); + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 101); +} + +TEST_F(ObTestRedoFill, parallel_logging_fill_from_all_list_ALL_FILLED_OTHERS_REMAIN) +{ + set_parallel_logging(true); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,99,98,99}; + return epochs[i]; + })); + EXPECT_CALL(mdo_, fill_log(_)) + .Times(1) + .WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 100; + ctx.cur_epoch_ = 98; + return OB_SUCCESS; + })); + EXPECT_EQ(OB_EAGAIN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99); // consume done, the cur_epoch point to epoch_to + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], INT64_MAX); +} + +TEST_F(ObTestRedoFill, parallel_logging_fill_from_one_list_OTHERS_IS_EMPTY) +{ + set_parallel_logging(true); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,INT64_MAX,INT64_MAX,INT64_MAX}; + return epochs[i]; + })); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_log(_)).Times(1).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 123; + ctx.cur_epoch_ = 100; + return OB_SUCCESS; + })); + } + EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); // consume done, the cur_epoch point to epoch_to + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 123); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 0); + EXPECT_TRUE(mdo_.fill_ctx_.is_all_filled_); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], INT64_MAX); +} + +TEST_F(ObTestRedoFill, serial_logging_fill_from_one_list_OTHERS_IS_EMPTY) +{ + set_parallel_logging(false); + callback_mgr_.need_merge_ = false; + // 4 list + extend_callback_lists_(3); + EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4)); + EXPECT_CALL(mdo_, get_log_epoch(_)) + .Times(AtLeast(1)) + .WillRepeatedly(Invoke([](int i){ + int64_t epochs[] = {100,INT64_MAX,INT64_MAX,INT64_MAX}; + return epochs[i]; + })); + { + InSequence s1; + EXPECT_CALL(mdo_, fill_log(_)).Times(1).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){ + ctx.fill_count_ += 123; + ctx.cur_epoch_ = 100; + return OB_SUCCESS; + })); + } + EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func)); + EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100); + EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX); + EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); // consume done, the cur_epoch point to epoch_to + EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 123); + EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 0); + EXPECT_TRUE(mdo_.fill_ctx_.is_all_filled_); + EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], INT64_MAX); +} + +} // memtable +} // oceanbase + +int main(int argc, char **argv) +{ + const char *log_name = "test_trans_callback_mgr_fill_redo.log"; + system("rm -rf test_trans_callback_mgr_fill_redo.log*"); + ObLogger &logger = ObLogger::get_logger(); + logger.set_file_name(log_name, true, false, + log_name, + log_name, + log_name); + logger.set_log_level(OB_LOG_LEVEL_DEBUG); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/unittest/storage/tx/test_undo_action.cpp b/unittest/storage/tx/test_undo_action.cpp new file mode 100644 index 0000000000..f19162786b --- /dev/null +++ b/unittest/storage/tx/test_undo_action.cpp @@ -0,0 +1,86 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include "storage/tx/ob_trans_define.h" +#include +#include "lib/oblog/ob_log.h" +namespace oceanbase +{ +using namespace transaction; +namespace unittest +{ +struct TestUndoAction : public ::testing::Test +{ + virtual void SetUp() {} + virtual void TearDown() {} +}; +TEST_F(TestUndoAction, valid) +{ + ObUndoAction a1(ObTxSEQ(100, 0), ObTxSEQ(1, 1)); + EXPECT_FALSE(a1.is_valid()); + ObUndoAction a2(ObTxSEQ(100, 1), ObTxSEQ(100, 1)); + EXPECT_FALSE(a2.is_valid()); + ObUndoAction a3(ObTxSEQ(100, 0), ObTxSEQ(100, 0)); + EXPECT_FALSE(a3.is_valid()); + ObUndoAction a4(ObTxSEQ(100, 0), ObTxSEQ(100, 1)); + EXPECT_FALSE(a4.is_valid()); + ObUndoAction a5(ObTxSEQ(100, 1), ObTxSEQ(100, 0)); + EXPECT_FALSE(a5.is_valid()); + ObUndoAction a6(ObTxSEQ(100, 1), ObTxSEQ(1, 0)); + EXPECT_FALSE(a6.is_valid()); + ObUndoAction a7(ObTxSEQ(100, 1), ObTxSEQ(1, 1)); + EXPECT_TRUE(a7.is_valid()); + ObUndoAction a8(ObTxSEQ(100, 0), ObTxSEQ(1, 0)); + EXPECT_TRUE(a8.is_valid()); +} +TEST_F(TestUndoAction, contain) +{ + ObUndoAction a1(ObTxSEQ(100,1), ObTxSEQ(1, 1)); + ObUndoAction a2(ObTxSEQ(99,1), ObTxSEQ(1, 1)); + EXPECT_TRUE(a1.is_contain(a2)); + EXPECT_FALSE(a2.is_contain(a1)); + ObUndoAction a3(ObTxSEQ(100,0), ObTxSEQ(1, 0)); + ObUndoAction a4(ObTxSEQ(99,0), ObTxSEQ(1, 0)); + EXPECT_TRUE(a3.is_contain(a4)); + EXPECT_FALSE(a4.is_contain(a3)); + ObUndoAction a5(ObTxSEQ(100,2), ObTxSEQ(1, 2)); + EXPECT_FALSE(a5.is_contain(a1)); + EXPECT_FALSE(a5.is_contain(a2)); + EXPECT_FALSE(a1.is_contain(a5)); + EXPECT_TRUE(a3.is_contain(a5)); + EXPECT_FALSE(a4.is_contain(a5)); +} + +TEST_F(TestUndoAction, contain_point) +{ + ObUndoAction a1(ObTxSEQ(100,1), ObTxSEQ(1, 1)); + ObUndoAction a3(ObTxSEQ(100,0), ObTxSEQ(1, 0)); + EXPECT_TRUE(a3.is_contain(ObTxSEQ(50, 1))); + EXPECT_TRUE(a1.is_contain(ObTxSEQ(50, 1))); + EXPECT_FALSE(a1.is_contain(ObTxSEQ(50, 0))); + EXPECT_FALSE(a1.is_contain(ObTxSEQ(50, 2))); + EXPECT_TRUE(a3.is_contain(ObTxSEQ(50, 0))); +} +} // unittest +} //oceanbase +using namespace oceanbase; +using namespace transaction; +int main(int argc, char **argv) +{ + int ret = 1; + ObLogger &logger = ObLogger::get_logger(); + logger.set_file_name("test_undo_action.log", true); + logger.set_log_level(OB_LOG_LEVEL_INFO); + testing::InitGoogleTest(&argc, argv); + ret = RUN_ALL_TESTS(); + return ret; +} diff --git a/unittest/storage/tx_table/test_tx_ctx_table.cpp b/unittest/storage/tx_table/test_tx_ctx_table.cpp index 0ab18a4404..4fdd650961 100644 --- a/unittest/storage/tx_table/test_tx_ctx_table.cpp +++ b/unittest/storage/tx_table/test_tx_ctx_table.cpp @@ -248,6 +248,7 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr) context.store_ctx_ = &store_ctx; context.allocator_ = &allocator; context.stmt_allocator_ = &allocator; + context.merge_scn_.convert_from_ts(996); context.is_inited_ = true; ObDatumRange key_range; @@ -274,6 +275,8 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr) ctx1.is_inited_ = true; ctx1.ls_id_ = ls_id; ctx1.exec_info_.max_applying_log_ts_.convert_from_ts(1); + ctx1.replay_completeness_.set(true); + ctx1.rec_log_ts_.convert_from_ts(996); ObTxData data1; // ctx1.tx_data_ = &data1; ctx1.ctx_tx_data_.test_init(data1, &ls_tx_ctx_mgr_); @@ -285,6 +288,8 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr) ctx2.is_inited_ = true; ctx2.ls_id_ = ls_id; ctx2.exec_info_.max_applying_log_ts_.convert_from_ts(2); + ctx2.replay_completeness_.set(true); + ctx2.rec_log_ts_.convert_from_ts(996); ObTxData data2; // ctx2.tx_data_ = &data2; ctx2.ctx_tx_data_.test_init(data2, &ls_tx_ctx_mgr_);