diff --git a/deps/oblib/src/lib/allocator/ob_slice_alloc.h b/deps/oblib/src/lib/allocator/ob_slice_alloc.h index 10726461f..c76b5b2b9 100644 --- a/deps/oblib/src/lib/allocator/ob_slice_alloc.h +++ b/deps/oblib/src/lib/allocator/ob_slice_alloc.h @@ -202,7 +202,7 @@ public: struct Item { public: - Item(Host* host): MAGIC_CODE_(ITEM_MAGIC_CODE), host_(host) {} + Item(Host *host): MAGIC_CODE_(ITEM_MAGIC_CODE), host_(host) {} ~Item(){} uint32_t MAGIC_CODE_; ObBlockSlicer* host_; @@ -353,6 +353,7 @@ public: } } } + return NULL == ret? NULL: (void*)(ret + 1); #endif } diff --git a/deps/oblib/src/lib/oblog/ob_log_module.h b/deps/oblib/src/lib/oblog/ob_log_module.h index 01153bedd..6142341f9 100644 --- a/deps/oblib/src/lib/oblog/ob_log_module.h +++ b/deps/oblib/src/lib/oblog/ob_log_module.h @@ -70,6 +70,7 @@ DEFINE_LOG_SUB_MOD(STANDBY) // primary and standby cluster DEFINE_LOG_SUB_MOD(REASY) // libreasy DEFINE_LOG_SUB_MOD(COORDINATOR) // leader coordinator DEFINE_LOG_SUB_MOD(OBTRACE) // trace +DEFINE_LOG_SUB_MOD(MVCC) // concurrency_control LOG_MOD_END(ROOT) //statement of WRS's sub_modules @@ -413,6 +414,8 @@ LOG_MOD_END(PL) #define _COORDINATOR_LOG(level, _fmt_, args...) _OB_MOD_LOG(COORDINATOR, level, _fmt_, ##args) #define OBTRACE_LOG(level, info_string, args...) OB_MOD_LOG(OBTRACE, level, info_string, ##args) #define _OBTRACE_LOG(level, _fmt_, args...) _OB_MOD_LOG(OBTRACE, level, _fmt_, ##args) +#define MVCC_LOG(level, info_string, args...) OB_MOD_LOG(MVCC, level, info_string, ##args) +#define _MVCC_LOG(level, _fmt_, args...) _OB_MOD_LOG(MVCC, level, _fmt_, ##args) //dfine ParMod_SubMod_LOG #define WRS_CLUSTER_LOG(level, info_string, args...) OB_SUB_MOD_LOG(WRS, CLUSTER, level, \ diff --git a/mittest/mtlenv/mock_tenant_module_env.h b/mittest/mtlenv/mock_tenant_module_env.h index c1e7d2d8f..e21eec6d1 100644 --- a/mittest/mtlenv/mock_tenant_module_env.h +++ b/mittest/mtlenv/mock_tenant_module_env.h @@ -70,6 +70,8 @@ #include "share/scn.h" #include "mock_gts_source.h" #include "storage/blocksstable/ob_shared_macro_block_manager.h" +#include "storage/concurrency_control/ob_multi_version_garbage_collector.h" +#include "storage/tx/wrs/ob_tenant_weak_read_service.h" namespace oceanbase { @@ -79,6 +81,7 @@ namespace storage { using namespace transaction; using namespace logservice; +using namespace concurrency_control; int64_t ObTenantMetaMemMgr::cal_adaptive_bucket_num() { @@ -650,6 +653,7 @@ int MockTenantModuleEnv::init() MTL_BIND2(mtl_new_default, storage::ObTenantSSTableMergeInfoMgr::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, storage::ObTenantFreezeInfoMgr::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObSharedMacroBlockMgr::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); + MTL_BIND2(mtl_new_default, ObMultiVersionGarbageCollector::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); } if (OB_FAIL(ret)) { diff --git a/mittest/mtlenv/storage/test_memtable_v2.cpp b/mittest/mtlenv/storage/test_memtable_v2.cpp index 47f2d0a46..e4500cf39 100644 --- a/mittest/mtlenv/storage/test_memtable_v2.cpp +++ b/mittest/mtlenv/storage/test_memtable_v2.cpp @@ -388,7 +388,20 @@ public: store_ctx->mvcc_acc_ctx_.abs_lock_timeout_ = abs_expire_time; store_ctx->mvcc_acc_ctx_.tx_scn_ = ObSequence::inc_and_get_max_seq_no(); } - + void start_pdml_stmt(ObStoreCtx *store_ctx, + const share::SCN snapshot_scn, + const int64_t read_seq_no, + const int64_t expire_time = 10000000000) + { + ObSequence::inc(); + 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_ = 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_ = ObSequence::inc_and_get_max_seq_no(); + } void print_callback(ObStoreCtx *wtx) { TRANS_LOG(INFO, "========== START PRINT CALLBACK ===========", K(wtx->mvcc_acc_ctx_.tx_id_)); @@ -2963,6 +2976,47 @@ TEST_F(TestMemtableV2, test_fast_commit_with_no_delay_cleanout) memtable->destroy(); } +TEST_F(TestMemtableV2, test_seq_set_violation) +{ + int ret = OB_SUCCESS; + ObMemtable *memtable = create_memtable(); + + TRANS_LOG(INFO, "######## CASE1: write row into memtable"); + ObDatumRowkey rowkey; + ObStoreRow write_row; + 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*/ + rowkey, + write_row2)); + + ObTransID write_tx_id = ObTransID(1); + ObStoreCtx *wtx = start_tx(write_tx_id); + + int64_t read_seq_no = 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*/); + EXPECT_EQ(OB_SUCCESS, (ret = memtable->set(*wtx, + tablet_id_.id(), + read_info_, + columns_, + write_row))); + + start_pdml_stmt(wtx, scn_3000, read_seq_no, 1000000000/*expire_time*/); + EXPECT_EQ(OB_ERR_PRIMARY_KEY_DUPLICATE, (ret = memtable->set(*wtx, + tablet_id_.id(), + read_info_, + columns_, + write_row))); + memtable->destroy(); +} + + } // namespace unittest namespace storage @@ -3000,10 +3054,11 @@ int ObTxCtxTable::release_ref_() namespace memtable { int ObMemtable::lock_row_on_frozen_stores_(ObStoreCtx &, + const ObTxNodeArg &, const ObMemtableKey *, ObMvccRow *, const storage::ObTableReadInfo &read_info, - ObStoreRowLockState &lock_state) + ObMvccWriteResult &) { if (unittest::TestMemtableV2::is_sstable_contains_lock_) { return OB_TRY_LOCK_ROW_CONFLICT; diff --git a/mittest/mtlenv/storage/test_multi_version_sstable_merge.cpp b/mittest/mtlenv/storage/test_multi_version_sstable_merge.cpp index 9c9e7a799..3654f2a31 100644 --- a/mittest/mtlenv/storage/test_multi_version_sstable_merge.cpp +++ b/mittest/mtlenv/storage/test_multi_version_sstable_merge.cpp @@ -81,21 +81,7 @@ int clear_tx_data(ObTxDataTable *tx_data_table) ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "tx data memtable is nullptr.", KR(ret), K(memtable_handles[i])); } else { - for (int64_t i = 0; OB_SUCC(ret) && i < 10; i++) { - tx_id = i; - if (tx_data_memtable->contain_tx_data(tx_id)) { - ObTxData *existed_tx_data = nullptr; - if (OB_FAIL(tx_data_memtable->get_tx_data(tx_id.tx_id_, existed_tx_data))) { - STORAGE_LOG(WARN, "get tx data from tx data memtable failed.", KR(ret), K(i), KPC(tx_data_memtable)); - } else if (OB_ISNULL(existed_tx_data)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "existed tx data is unexpected nullptr", KR(ret), - KPC(tx_data_memtable)); - } else if (OB_FAIL(tx_data_memtable->remove(tx_id.tx_id_))) { - STORAGE_LOG(ERROR, "remove tx data from tx data memtable failed.", KR(ret), K(tx_id), KPC(tx_data_memtable)); - } - } - } + tx_data_memtable->TEST_reset_tx_data_map_(); } } } @@ -1060,8 +1046,9 @@ TEST_F(TestMultiVersionMerge, test_merge_with_multi_trans) for (int64_t i = 1; i <= 4; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + tx_data = tx_data_guard.tx_data(); // fill in data tx_data->tx_id_ = tx_id; @@ -1080,7 +1067,6 @@ TEST_F(TestMultiVersionMerge, test_merge_with_multi_trans) } ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -1229,8 +1215,9 @@ TEST_F(TestMultiVersionMerge, test_merge_with_multi_trans_can_compact) for (int64_t i = 1; i <= 5; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -1249,7 +1236,6 @@ TEST_F(TestMultiVersionMerge, test_merge_with_multi_trans_can_compact) } ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -1398,8 +1384,9 @@ TEST_F(TestMultiVersionMerge, test_merge_with_multi_trans_can_not_compact) for (int64_t i = 1; i <= 5; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -1419,7 +1406,6 @@ TEST_F(TestMultiVersionMerge, test_merge_with_multi_trans_can_not_compact) } ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -1654,8 +1640,9 @@ TEST_F(TestMultiVersionMerge, test_merge_with_macro_reused_without_shadow) for (int64_t i = 1; i <= 4; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -1665,7 +1652,6 @@ TEST_F(TestMultiVersionMerge, test_merge_with_macro_reused_without_shadow) tx_data->state_ = ObTxData::COMMIT; ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -1868,8 +1854,9 @@ TEST_F(TestMultiVersionMerge, test_merge_with_greater_multi_version_and_uncommit for (int64_t i = 1; i <= 4; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -1879,7 +1866,6 @@ TEST_F(TestMultiVersionMerge, test_merge_with_greater_multi_version_and_uncommit tx_data->state_ = ObTxData::COMMIT; ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } prepare_merge_context(MINOR_MERGE, false, trans_version_range, merge_context); @@ -1997,8 +1983,9 @@ TEST_F(TestMultiVersionMerge, test_merge_with_ghost_row) for (int64_t i = 1; i <= 4; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -2015,7 +2002,6 @@ TEST_F(TestMultiVersionMerge, test_merge_with_ghost_row) } ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -2404,8 +2390,9 @@ TEST_F(TestMultiVersionMerge, rowkey_cross_macro_with_last_shadow_version_less_t for (int64_t i = 1; i < 3; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -2415,7 +2402,6 @@ TEST_F(TestMultiVersionMerge, rowkey_cross_macro_with_last_shadow_version_less_t tx_data->state_ = ObTxData::COMMIT; ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -2633,8 +2619,9 @@ TEST_F(TestMultiVersionMerge, rowkey_cross_macro_without_open_next_macro) transaction::ObTransID tx_id; for (int64_t i = 1; i < 3; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; tx_data->commit_version_.convert_for_tx(INT64_MAX); @@ -2642,7 +2629,6 @@ TEST_F(TestMultiVersionMerge, rowkey_cross_macro_without_open_next_macro) tx_data->end_scn_.convert_for_tx(100); tx_data->state_ = ObTxData::ABORT; ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; @@ -2880,8 +2866,9 @@ TEST_F(TestMultiVersionMerge, test_merge_base_iter_have_ghost_row) for (int64_t i = 1; i <= 4; i++) { tx_id = i; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table->alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -2898,7 +2885,6 @@ TEST_F(TestMultiVersionMerge, test_merge_base_iter_have_ghost_row) } ASSERT_EQ(OB_SUCCESS, tx_data_table->insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } ObVersionRange trans_version_range; diff --git a/mittest/mtlenv/tablelock/table_lock_tx_common_env.h b/mittest/mtlenv/tablelock/table_lock_tx_common_env.h index 0ba50c94c..d0b409790 100644 --- a/mittest/mtlenv/tablelock/table_lock_tx_common_env.h +++ b/mittest/mtlenv/tablelock/table_lock_tx_common_env.h @@ -137,6 +137,7 @@ void MockTxEnv::get_store_ctx(MyTxCtx &my_ctx, snapshot.version_ = share::SCN::base_scn(); int64_t tx_lock_timeout = 0; ObTxTableGuard tx_table_guard; + concurrent_control::ObWriteFlag write_flag; tx_table_guard.tx_table_ = tx_table; store_ctx.ls_id_ = my_ctx.tx_ctx_.ls_id_; store_ctx.mvcc_acc_ctx_.init_write(my_ctx.tx_ctx_, @@ -147,7 +148,8 @@ void MockTxEnv::get_store_ctx(MyTxCtx &my_ctx, tx_table_guard, snapshot, timeout, - timeout); + timeout, + write_flag); store_ctx.replay_log_scn_ = share::SCN::base_scn(); } diff --git a/mittest/mtlenv/tablelock/test_lock_memtable.cpp b/mittest/mtlenv/tablelock/test_lock_memtable.cpp index db76e8535..5f3bbe178 100644 --- a/mittest/mtlenv/tablelock/test_lock_memtable.cpp +++ b/mittest/mtlenv/tablelock/test_lock_memtable.cpp @@ -15,6 +15,7 @@ #include #define protected public #define private public +#define UNITTEST #include "mtlenv/mock_tenant_module_env.h" #include "storage/tablelock/ob_lock_memtable.h" diff --git a/mittest/mtlenv/tablelock/test_obj_lock.cpp b/mittest/mtlenv/tablelock/test_obj_lock.cpp index bc819df48..92ecf899c 100644 --- a/mittest/mtlenv/tablelock/test_obj_lock.cpp +++ b/mittest/mtlenv/tablelock/test_obj_lock.cpp @@ -15,6 +15,7 @@ #include #define protected public #define private public +#define UNITTEST #include "mtlenv/mock_tenant_module_env.h" #include "storage/tablelock/ob_obj_lock.h" diff --git a/mittest/mtlenv/tablelock/test_obj_lock_map.cpp b/mittest/mtlenv/tablelock/test_obj_lock_map.cpp index d6f3629a4..cc973883d 100644 --- a/mittest/mtlenv/tablelock/test_obj_lock_map.cpp +++ b/mittest/mtlenv/tablelock/test_obj_lock_map.cpp @@ -14,6 +14,7 @@ #include #define protected public #define private public +#define UNITTEST #include "mtlenv/mock_tenant_module_env.h" #include "storage/tablelock/ob_obj_lock.h" diff --git a/mittest/mtlenv/tablelock/test_table_lock_flush.cpp b/mittest/mtlenv/tablelock/test_table_lock_flush.cpp index 94829c648..9c80183b9 100644 --- a/mittest/mtlenv/tablelock/test_table_lock_flush.cpp +++ b/mittest/mtlenv/tablelock/test_table_lock_flush.cpp @@ -128,6 +128,7 @@ static void mock_store_ctx(ObStoreCtx &store_ctx, ObTxTableGuard tx_guard; tx_guard.tx_table_ = (ObTxTable*)0x01; ObTxSnapshot snapshot; + concurrent_control::ObWriteFlag write_flag; snapshot.version_ = share::SCN::base_scn(); store_ctx.mvcc_acc_ctx_.init_write(part_ctx, part_ctx.mt_ctx_, @@ -137,7 +138,8 @@ static void mock_store_ctx(ObStoreCtx &store_ctx, tx_guard, snapshot, 50000, - 10000); + 10000, + write_flag); } TEST_F(TestTableLockFlush, checkpoint) diff --git a/mittest/mtlenv/test_tx_data_table.cpp b/mittest/mtlenv/test_tx_data_table.cpp index 5f29c8309..544b3d06d 100644 --- a/mittest/mtlenv/test_tx_data_table.cpp +++ b/mittest/mtlenv/test_tx_data_table.cpp @@ -14,6 +14,7 @@ #define protected public #define private public +#define UNITTEST #include #include @@ -51,9 +52,21 @@ class MockTxDataTable; class MockTxTable; static const uint64_t TEST_TENANT_ID = 1; -// shrink select interval to push more points in cur_commit_scns +// shrink select interval to push more points in cur_commit_versions // then the code will merge commit versions array with step_len larger than 1 -int64_t ObTxDataMemtableScanIterator::PERIODICAL_SELECT_INTERVAL_NS = 10LL; +int64_t ObTxDataMemtable::PERIODICAL_SELECT_INTERVAL_NS = 10LL; + +int ObTxDataMemtable::get_past_commit_versions_(ObCommitVersionsArray &past_commit_versions) +{ + int ret = OB_SUCCESS; + return ret; +} + +int ObTxDataTable::get_recycle_scn(SCN &recycle_scn) +{ + recycle_scn = SCN::max_scn(); + return OB_SUCCESS; +} class MockTxDataMemtableMgr : public ObTxDataMemtableMgr { @@ -166,8 +179,6 @@ public: void do_repeat_insert_test(); - void do_multiple_init_iterator_test(); - void do_print_leak_slice_test(); @@ -179,7 +190,7 @@ private: void test_serialize_with_action_cnt_(int cnt); - void generate_past_commit_scn_(ObCommitSCNsArray &past_commit_scns); + void generate_past_commit_version_(ObCommitVersionsArray &past_commit_versions); void set_freezer_(); @@ -189,15 +200,11 @@ private: ObTxDataMemtable *&freezing_memtable, ObTxDataMemtable *&active_memtable); - void check_commit_scn_row_(ObTxDataMemtableScanIterator &iter, ObTxDataMemtable *freezing_memtable); - - void make_freezing_to_frozen(ObTxDataMemtableMgr *memtable_mgr); - void test_serialize_with_action_cnt(int cnt); void insert_rollback_tx_data_(); - void test_commit_scns_serialize_(); + void test_commit_versions_serialize_(); void fake_ls_(ObLS &ls); @@ -230,12 +237,14 @@ void TestTxDataTable::insert_tx_data_() while (true) { int64_t int_tx_id = 0; - if ((int_tx_id = ATOMIC_AAF(&tx_data_num, -1)) < 0) { break; } + if ((int_tx_id = ATOMIC_AAF(&tx_data_num, -1)) <= 0) { break; } tx_id = int_tx_id; bool is_abort_tx = int_tx_id % 5 == 0; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + tx_data_guard.reset(); + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -252,7 +261,6 @@ void TestTxDataTable::insert_tx_data_() } ASSERT_EQ(OB_SUCCESS, tx_data_table_.insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } } @@ -267,8 +275,10 @@ void TestTxDataTable::insert_rollback_tx_data_() ASSERT_NE(nullptr, memtable); for (int i = 0; i < 200; i++) { + ObTxDataGuard tx_data_guard; ObTxData *tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -283,12 +293,6 @@ void TestTxDataTable::insert_rollback_tx_data_() tx_data->state_ = ObTxData::RUNNING; ASSERT_EQ(OB_SUCCESS, tx_data_table_.insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); - { - ObTxDataGuard guard; - memtable->get_tx_data(tx_id, guard); - ASSERT_EQ(max_end_scn, guard.tx_data().end_scn_); - } } } @@ -300,8 +304,9 @@ void TestTxDataTable::insert_abort_tx_data_() tx_id = INT64_MAX - 3; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -311,15 +316,14 @@ void TestTxDataTable::insert_abort_tx_data_() tx_data->state_ = ObTxData::ABORT; ASSERT_EQ(OB_SUCCESS, tx_data_table_.insert(tx_data)); - ASSERT_EQ(nullptr, tx_data); } -void TestTxDataTable::generate_past_commit_scn_(ObCommitSCNsArray &past_commit_scns) +void TestTxDataTable::generate_past_commit_version_(ObCommitVersionsArray &past_commit_versions) { share::SCN start_scn = share::SCN::minus(insert_start_scn, 300LL * ONE_SEC_NS); share::SCN commit_version = share::SCN::plus(start_scn, 2LL * ONE_SEC_NS); for (int i = 0; i < 500; i++) { - past_commit_scns.array_.push_back(ObCommitSCNsArray::Node(start_scn, commit_version)); + past_commit_versions.array_.push_back(ObCommitVersionsArray::Node(start_scn, commit_version)); start_scn = share::SCN::plus(start_scn, 1LL * ONE_SEC_NS + (rand64(ObTimeUtil::current_time_ns()) % ONE_SEC_NS)); commit_version = share::SCN::plus(std::max(commit_version, start_scn), (rand64(ObTimeUtil::current_time_ns()) % (2LL * ONE_SEC_NS))); } @@ -370,177 +374,67 @@ void TestTxDataTable::check_freeze_(ObTxDataMemtableMgr *memtable_mgr, freezing_memtable->set_state(ObTxDataMemtable::State::FROZEN); } -void TestTxDataTable::check_commit_scn_row_(ObTxDataMemtableScanIterator &iter, ObTxDataMemtable *freezing_memtable) -{ - // int ret = OB_SUCCESS; - ObCommitSCNsArray cur_commit_scns ; - ObCommitSCNsArray past_commit_scns; - ObCommitSCNsArray merged_commit_scns; - auto &cur_array = cur_commit_scns.array_; - auto &past_array = past_commit_scns.array_; - auto &merged_array = merged_commit_scns.array_; - share::SCN max_commit_version = share::SCN::min_scn(); - share::SCN max_start_scn = share::SCN::min_scn(); - - // check sort commit version result - { - ASSERT_EQ(OB_SUCCESS, freezing_memtable->do_sort_by_start_scn_()); - share::SCN pre_start_scn = share::SCN::min_scn(); - ObTxData *last_commit_tx_data = nullptr; - auto cur_node = freezing_memtable->sort_list_head_.next_; - ASSERT_NE(nullptr, cur_node); - int64_t cnt = 0; - while (nullptr != cur_node) { - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); - ASSERT_GE(tx_data->start_scn_, pre_start_scn); - if (ObTxData::COMMIT == tx_data->state_) { - last_commit_tx_data = tx_data; - max_commit_version = std::max(max_commit_version, tx_data->commit_version_); - max_start_scn = std::max(max_start_scn, tx_data->start_scn_); - } - STORAGETEST_LOG(DEBUG, - "check_commit_scn_row", - KPC(tx_data), - KTIME(tx_data->start_scn_.convert_to_ts()), - KTIME(tx_data->end_scn_.convert_to_ts())); - pre_start_scn = tx_data->start_scn_; - cur_node = cur_node->next_; - cnt++; - } - last_commit_tx_data->commit_version_ = share::SCN::plus(max_commit_version, 1); - max_commit_version = last_commit_tx_data->commit_version_; - ASSERT_EQ(cnt, inserted_cnt); - fprintf(stdout, "total insert %ld tx data\n", cnt); - - ASSERT_EQ(OB_SUCCESS, iter.fill_in_cur_commit_scns_(cur_commit_scns)); - STORAGETEST_LOG(INFO, "cur_commit_scns count", K(cur_commit_scns.array_.count())); - ASSERT_NE(0, cur_commit_scns.array_.count()); - for (int i = 1; i < cur_array.count() - 1; i++) { - ASSERT_GE(cur_array.at(i).start_scn_, - share::SCN::plus(cur_array.at(i - 1).start_scn_, iter.PERIODICAL_SELECT_INTERVAL_NS)); - ASSERT_GE(cur_array.at(i).commit_version_, cur_array.at(i - 1).commit_version_); - } - int i = cur_array.count() - 1; - ASSERT_GE(cur_array.at(i).start_scn_, cur_array.at(i-1).start_scn_); - ASSERT_GE(cur_array.at(i).commit_version_, cur_array.at(i-1).commit_version_); - ASSERT_EQ(cur_array.at(i).start_scn_, max_start_scn); - ASSERT_EQ(cur_array.at(i).commit_version_, max_commit_version); - } - - // generate a fake past commit versions - { - generate_past_commit_scn_(past_commit_scns); - ASSERT_NE(0, past_commit_scns.array_.count()); - ASSERT_EQ(true, past_commit_scns.is_valid()); - } - - // check merged result - { - share::SCN recycle_scn = share::SCN::minus(insert_start_scn, 100LL * ONE_SEC_NS/*100 seconds*/); - ASSERT_EQ(OB_SUCCESS, iter.merge_cur_and_past_commit_verisons_(recycle_scn, cur_commit_scns, - past_commit_scns, - merged_commit_scns)); - for (int i = 0; i < merged_array.count(); i++) { - STORAGE_LOG(INFO, "print merged array", K(merged_array.at(i))); - } - ASSERT_EQ(true, merged_commit_scns.is_valid()); - fprintf(stdout, - "merge commit versions finish. past array count = %ld current array count = %ld merged array count = %ld\n", - past_array.count(), - cur_array.count(), - merged_array.count()); - } - - // check commit versions serialization and deserialization - { - int64_t m_size = merged_commit_scns.get_serialize_size(); - ObArenaAllocator allocator; - ObTxLocalBuffer buf_(allocator); - buf_.reserve(m_size); - - int64_t pos = 0; - ASSERT_EQ(OB_SUCCESS, merged_commit_scns.serialize(buf_.get_ptr(), m_size, pos)); - - // void *ptr = allocator_.alloc(sizeof(ObCommitSCNsArray)); - ObCommitSCNsArray deserialize_commit_scns; - pos = 0; - ASSERT_EQ(OB_SUCCESS, deserialize_commit_scns.deserialize(buf_.get_ptr(), m_size, pos)); - - const auto &deserialize_array = deserialize_commit_scns.array_; - ASSERT_EQ(merged_array.count(), deserialize_array.count()); - for (int i = 0; i < merged_commit_scns.array_.count(); i++) { - ASSERT_EQ(merged_array.at(i), deserialize_array.at(i)); - } - } - - share::SCN sstable_end_scn = share::SCN::min_scn(); - share::SCN upper_trans_scn = share::SCN::min_scn(); - tx_data_table_.calc_upper_trans_version_cache_.commit_scns_ = merged_commit_scns; - - // check the situation when sstable_end_scn is greater than the greatest start_scn in - // merged_array - { - sstable_end_scn = share::SCN::plus(max_start_scn, 1); - upper_trans_scn.set_max(); - ASSERT_EQ(OB_SUCCESS, - tx_data_table_.calc_upper_trans_scn_(sstable_end_scn, upper_trans_scn)); - ASSERT_EQ(max_commit_version, upper_trans_scn); - } - - // check the normal calculation - { - share::SCN second_max_start_scn = share::SCN::minus(merged_array.at(merged_array.count() - 2).start_scn_, 1); - for (int i = 0; i < 100; i++) { - sstable_end_scn = - share::SCN::minus(second_max_start_scn, rand64(ObTimeUtil::current_time_ns()) % (1100LL * ONE_SEC_NS)); - upper_trans_scn = share::SCN::max_scn(); - ASSERT_EQ(OB_SUCCESS, - tx_data_table_.calc_upper_trans_scn_(sstable_end_scn, upper_trans_scn)); - ASSERT_NE(SCN::max_scn(), upper_trans_scn); - ASSERT_NE(max_commit_version, upper_trans_scn); - } - } -} - void TestTxDataTable::do_basic_test() { - tx_data_table_.TEST_print_alloc_size_(); // init tx data table ASSERT_EQ(OB_SUCCESS, tx_data_table_.init(ls_)); set_freezer_(); ObTxDataMemtableMgr *memtable_mgr = tx_data_table_.get_memtable_mgr_(); init_memtable_mgr_(memtable_mgr); + fprintf(stdout, "start insert tx data\n"); insert_tx_data_(); + fprintf(stdout, "start insert rollback tx data\n"); insert_rollback_tx_data_(); + fprintf(stdout, "start insert abort tx data\n"); insert_abort_tx_data_(); + fprintf(stdout, "start freezing\n"); ObTxDataMemtable *freezing_memtable = nullptr; ObTxDataMemtable *active_memtable = nullptr; check_freeze_(memtable_mgr, freezing_memtable, active_memtable); inserted_cnt = freezing_memtable->get_tx_data_count(); - // sort tx data by trans id - ObTxDataMemtableScanIterator iter(tx_data_table_.get_read_schema().iter_param_); - ASSERT_EQ(OB_SUCCESS, iter.init(freezing_memtable)); + const int64_t range_cnt = 4; + ObSEArray range_array; + ASSERT_EQ(OB_SUCCESS, freezing_memtable->pre_process_for_merge()); + ASSERT_EQ(OB_SUCCESS, freezing_memtable->get_split_ranges(nullptr, nullptr, range_cnt, range_array)); + int64_t pre_range_end_key = 0; + for (int i = 0; i < range_cnt; i++) { + auto &range = range_array[i]; + int64_t start_key = 0; + int64_t end_key = 0; + ASSERT_EQ(OB_SUCCESS, range.get_start_key().get_obj_ptr()[0].get_int(start_key)); + ASSERT_EQ(OB_SUCCESS, range.get_end_key().get_obj_ptr()[0].get_int(end_key)); + ASSERT_EQ(pre_range_end_key, start_key); + ASSERT_GE(end_key, start_key); + pre_range_end_key = end_key; + } + + ObTxData *fake_tx_data = nullptr; // check sort result { transaction::ObTransID pre_tx_id = INT64_MIN; - auto cur_node = freezing_memtable->sort_list_head_.next_; - ASSERT_NE(nullptr, cur_node); + ObTxData *cur_tx_data = freezing_memtable->sort_list_head_.next_; + ASSERT_NE(nullptr, cur_tx_data); int64_t cnt = 0; - while (nullptr != cur_node) { - auto tx_id = ObTxData::get_tx_data_by_sort_list_node(cur_node)->tx_id_; - ASSERT_GT(tx_id.get_id(), pre_tx_id.get_id()); + while (nullptr != cur_tx_data) { + auto tx_id = cur_tx_data->tx_id_; + if (INT64_MAX == tx_id) { + fake_tx_data = cur_tx_data; + } + ASSERT_GE(tx_id.get_id(), pre_tx_id.get_id()); + pre_tx_id = tx_id; - cur_node = cur_node->next_; + cur_tx_data = cur_tx_data->sort_list_node_.next_; cnt++; } - ASSERT_EQ(inserted_cnt, cnt); - } - check_commit_scn_row_(iter, freezing_memtable); + // there is a fake tx data inserted into link hash map after pre-process for upper_trans_version + // calculation + ASSERT_EQ(inserted_cnt + 1, cnt); + } // free memtable freezing_memtable->reset(); @@ -555,15 +449,16 @@ void TestTxDataTable::do_undo_status_test() // the last undo action covers all the previous undo actions { ObTxData *tx_data = nullptr; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); tx_data->tx_id_ = rand(); - for (int i = 1; i <= 1000; i++) { + for (int i = 1; i <= 1001; i++) { transaction::ObUndoAction undo_action(10 * (i + 1), 10 * i); ASSERT_EQ(OB_SUCCESS, tx_data->add_undo_action(&tx_table_, undo_action)); } - ASSERT_EQ(1000 / 7 + 1, tx_data->undo_status_list_.undo_node_cnt_); + ASSERT_EQ(1000 / TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE + 1, tx_data->undo_status_list_.undo_node_cnt_); { transaction::ObUndoAction undo_action(10000000, 10); @@ -580,8 +475,9 @@ void TestTxDataTable::do_undo_status_test() // the last undo action covers eight previous undo actions // so the undo status just have one undo status node ObTxData *tx_data = nullptr; - tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); tx_data->tx_id_ = rand(); for (int i = 1; i <= 14; i++) { @@ -605,7 +501,9 @@ void TestTxDataTable::do_undo_status_test() void TestTxDataTable::test_serialize_with_action_cnt_(int cnt) { ObTxData *tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); tx_data->tx_id_ = transaction::ObTransID(269381); tx_data->commit_version_.convert_for_logservice(ObTimeUtil::current_time_ns()); tx_data->end_scn_.convert_for_logservice(ObTimeUtil::current_time_ns()); @@ -632,36 +530,40 @@ void TestTxDataTable::test_serialize_with_action_cnt_(int cnt) ASSERT_EQ(OB_SUCCESS, tx_data->serialize(buf, serialize_size, pos)); ObTxData *new_tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(new_tx_data)); + ObTxDataGuard new_tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(new_tx_data_guard)); + ASSERT_NE(nullptr, new_tx_data = new_tx_data_guard.tx_data()); new_tx_data->tx_id_ = transaction::ObTransID(269381); pos = 0; ASSERT_EQ(OB_SUCCESS, new_tx_data->deserialize(buf, serialize_size, pos, *tx_data_table_.get_slice_allocator())); ASSERT_TRUE(new_tx_data->equals_(*tx_data)); - tx_data_table_.free_tx_data(tx_data); - tx_data_table_.free_tx_data(new_tx_data); + tx_data->dec_ref(); + new_tx_data->dec_ref(); } void TestTxDataTable::do_tx_data_serialize_test() { - // init tx data table ASSERT_EQ(OB_SUCCESS, tx_data_table_.init(ls_)); + ObTxDataMemtableMgr *memtable_mgr = tx_data_table_.get_memtable_mgr_(); + set_freezer_(); + init_memtable_mgr_(memtable_mgr); test_serialize_with_action_cnt_(0); - test_serialize_with_action_cnt_(7); - test_serialize_with_action_cnt_(8); - test_serialize_with_action_cnt_(7 * 10000); - test_serialize_with_action_cnt_(7 * 10000 + 1); - test_commit_scns_serialize_(); + test_serialize_with_action_cnt_(TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE); + test_serialize_with_action_cnt_(TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE + 1); + test_serialize_with_action_cnt_(TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE * 100); + test_serialize_with_action_cnt_(TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE * 100 + 1); + test_commit_versions_serialize_(); } -void TestTxDataTable::test_commit_scns_serialize_() +void TestTxDataTable::test_commit_versions_serialize_() { - ObCommitSCNsArray cur_array; - ObCommitSCNsArray past_array; - ObCommitSCNsArray merged_array; - ObCommitSCNsArray deserialized_array; + ObCommitVersionsArray cur_array; + ObCommitVersionsArray past_array; + ObCommitVersionsArray merged_array; + ObCommitVersionsArray deserialized_array; share::SCN start_scn; start_scn.convert_for_logservice(ObTimeUtil::current_time_ns()); @@ -672,7 +574,7 @@ void TestTxDataTable::test_commit_scns_serialize_() STORAGE_LOG(INFO, "start generate past array"); for (int64_t i = 0; i < array_cnt; i++) { start_scn = share::SCN::plus(start_scn, (rand64(ObTimeUtil::current_time_ns()) % MOD)); - ObCommitSCNsArray::Node node(start_scn, share::SCN::plus(start_scn, (rand64(ObTimeUtil::current_time_ns()) % MOD))); + ObCommitVersionsArray::Node node(start_scn, share::SCN::plus(start_scn, (rand64(ObTimeUtil::current_time_ns()) % MOD))); STORAGE_LOG(INFO, "", K(node)); ASSERT_EQ(OB_SUCCESS, past_array.array_.push_back(node)); } @@ -681,15 +583,19 @@ void TestTxDataTable::test_commit_scns_serialize_() STORAGE_LOG(INFO, "start generate cur array"); for (int i = 0; i < array_cnt; i++) { start_scn = share::SCN::plus(start_scn, (rand64(ObTimeUtil::current_time_ns()) % MOD)); - ObCommitSCNsArray::Node node(start_scn, share::SCN::plus(start_scn, (rand64(ObTimeUtil::current_time_ns()) % MOD))); + ObCommitVersionsArray::Node node(start_scn, share::SCN::plus(start_scn, (rand64(ObTimeUtil::current_time_ns()) % MOD))); STORAGE_LOG(DEBUG, "", K(node)); ASSERT_EQ(OB_SUCCESS, cur_array.array_.push_back(node)); } ASSERT_EQ(true, cur_array.is_valid()); - ObTxDataMemtableScanIterator iter(tx_data_table_.get_read_schema().iter_param_); - ASSERT_EQ(OB_SUCCESS, iter.merge_cur_and_past_commit_verisons_(recycle_scn, cur_array, past_array, - merged_array)); + ObTxDataMemtableMgr *memtable_mgr = tx_data_table_.get_memtable_mgr_(); + ASSERT_NE(nullptr, memtable_mgr); + ObTableHandleV2 table_handle; + ASSERT_EQ(OB_SUCCESS, memtable_mgr->get_active_memtable(table_handle)); + ObTxDataMemtable *tx_data_memtable = nullptr; + ASSERT_EQ(OB_SUCCESS, table_handle.get_tx_data_memtable(tx_data_memtable)); + ASSERT_EQ(OB_SUCCESS, tx_data_memtable->merge_cur_and_past_commit_verisons_(recycle_scn, cur_array, past_array, merged_array)); ASSERT_EQ(true, merged_array.is_valid()); int64_t serialize_size = merged_array.get_serialize_size(); @@ -730,7 +636,9 @@ void TestTxDataTable::do_repeat_insert_test() { for (int i = 1; i <= 100; i++) { tx_id = transaction::ObTransID(269381); tx_data = nullptr; - ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data)); + ObTxDataGuard tx_data_guard; + ASSERT_EQ(OB_SUCCESS, tx_data_table_.alloc_tx_data(tx_data_guard)); + ASSERT_NE(nullptr, tx_data = tx_data_guard.tx_data()); // fill in data tx_data->tx_id_ = tx_id; @@ -753,45 +661,6 @@ void TestTxDataTable::do_repeat_insert_test() { } -void TestTxDataTable::do_multiple_init_iterator_test() -{ - ASSERT_EQ(OB_SUCCESS, tx_data_table_.init(ls_)); - set_freezer_(); - - ObTxDataMemtableMgr *memtable_mgr = tx_data_table_.get_memtable_mgr_(); - init_memtable_mgr_(memtable_mgr); - tx_data_num = 10240; - insert_tx_data_(); - - ObTxDataMemtable *freezing_memtable = nullptr; - ObTxDataMemtable *active_memtable = nullptr; - check_freeze_(memtable_mgr, freezing_memtable, active_memtable); - inserted_cnt = freezing_memtable->get_tx_data_count(); - - // sort tx data by trans id - { - ObTxDataMemtableScanIterator iter1(tx_data_table_.get_read_schema().iter_param_); - ASSERT_EQ(OB_SUCCESS, iter1.init(freezing_memtable)); - - ObTxDataMemtableScanIterator iter2(tx_data_table_.get_read_schema().iter_param_); - ASSERT_EQ(OB_STATE_NOT_MATCH, iter2.init(freezing_memtable)); - - // iter1 can init succeed because of the reset function in init() - ASSERT_EQ(OB_SUCCESS, iter1.init(freezing_memtable)); - - // iter2 still can not init - ASSERT_EQ(OB_STATE_NOT_MATCH, iter2.init(freezing_memtable)); - - iter1.reset(); - // now iter2 can successfully init - ASSERT_EQ(OB_SUCCESS, iter2.init(freezing_memtable)); - } - - // iter3 can successfully init due to the destruct function of iterator - ObTxDataMemtableScanIterator iter3(tx_data_table_.get_read_schema().iter_param_); - ASSERT_EQ(OB_SUCCESS, iter3.init(freezing_memtable)); -} - void TestTxDataTable::fake_ls_(ObLS &ls) { ls.ls_meta_.tenant_id_ = 1; @@ -852,8 +721,6 @@ TEST_F(TestTxDataTable, serialize_test) { do_tx_data_serialize_test(); } // TEST_F(TestTxDataTable, print_leak_slice) { do_print_leak_slice_test(); } -// TEST_F(TestTxDataTable, iterate_init_test) { do_multiple_init_iterator_test(); } - } // namespace storage } // namespace oceanbase @@ -861,7 +728,8 @@ TEST_F(TestTxDataTable, serialize_test) { do_tx_data_serialize_test(); } int main(int argc, char **argv) { int ret = 1; - system("rm -f test_tx_data_table.log"); + system("rm -f test_tx_data_table.log*"); + system("rm -fr run_*"); ObLogger &logger = ObLogger::get_logger(); logger.set_file_name("test_tx_data_table.log", true); logger.set_log_level(OB_LOG_LEVEL_INFO); diff --git a/mittest/simple_server/CMakeLists.txt b/mittest/simple_server/CMakeLists.txt index 410f545cf..00cafbe96 100644 --- a/mittest/simple_server/CMakeLists.txt +++ b/mittest/simple_server/CMakeLists.txt @@ -33,3 +33,6 @@ ob_unittest_observer(test_lock_table_persistence test_lock_table_persistence.cpp ob_unittest_observer(test_tx_recover test_tx_recovery.cpp) ob_unittest_observer(test_tx_recover2 test_tx_recovery2.cpp) ob_unittest_observer(test_tx_recover3 test_tx_recovery3.cpp) +ob_unittest_observer(test_big_tx_data test_big_tx_data.cpp) +ob_unittest_observer(test_fast_commit_report fast_commit_report.cpp) +ob_unittest_observer(test_mvcc_gc test_mvcc_gc.cpp) diff --git a/mittest/simple_server/env/ob_simple_cluster_test_base.cpp b/mittest/simple_server/env/ob_simple_cluster_test_base.cpp index a2ba416f8..3ec719c4b 100644 --- a/mittest/simple_server/env/ob_simple_cluster_test_base.cpp +++ b/mittest/simple_server/env/ob_simple_cluster_test_base.cpp @@ -80,11 +80,15 @@ std::string ObSimpleClusterTestBase::env_prefix_; std::string ObSimpleClusterTestBase::curr_dir_; bool ObSimpleClusterTestBase::enable_env_warn_log_ = false; -ObSimpleClusterTestBase::ObSimpleClusterTestBase(const std::string &env_prefix) +ObSimpleClusterTestBase::ObSimpleClusterTestBase(const std::string &env_prefix, + const char *log_disk_size, + const char *memory_limit) { if (cluster_ == nullptr) { env_prefix_ = env_prefix + "_test_data"; //+ std::to_string(ObTimeUtility::current_time()) + "_"; - cluster_ = std::make_shared(env_prefix_); + cluster_ = std::make_shared(env_prefix_, + log_disk_size, + memory_limit); curr_dir_ = get_current_dir_name(); } } @@ -180,7 +184,9 @@ int ObSimpleClusterTestBase::close() return ret; } -int ObSimpleClusterTestBase::create_tenant(const char *tenant_name) +int ObSimpleClusterTestBase::create_tenant(const char *tenant_name, + const char *memory_size, + const char *log_disk_size) { SERVER_LOG(INFO, "create tenant start"); int32_t log_level; @@ -216,7 +222,8 @@ int ObSimpleClusterTestBase::create_tenant(const char *tenant_name) { ObSqlString sql; if (OB_FAIL(ret)) { - } else if (OB_FAIL(sql.assign_fmt("create resource unit box_ym_%s max_cpu 2, memory_size '2G', log_disk_size='2G';", tenant_name))) { + } else if (OB_FAIL(sql.assign_fmt("create resource unit box_ym_%s max_cpu 8, memory_size '%s', log_disk_size='%s';", + tenant_name, memory_size, log_disk_size))) { SERVER_LOG(WARN, "create_tenant", K(ret)); } else if (OB_FAIL(sql_proxy.write(sql.ptr(), affected_rows))) { SERVER_LOG(WARN, "create_tenant", K(ret)); diff --git a/mittest/simple_server/env/ob_simple_cluster_test_base.h b/mittest/simple_server/env/ob_simple_cluster_test_base.h index 743665a90..ed3a52d22 100644 --- a/mittest/simple_server/env/ob_simple_cluster_test_base.h +++ b/mittest/simple_server/env/ob_simple_cluster_test_base.h @@ -30,7 +30,9 @@ class ObSimpleClusterTestBase : public testing::Test public: static const int64_t TRANS_TIMEOUT = 5 * 1000 * 1000; // set_bootstrap_and_create_tenant_warn_log 默认bootstrap和创建租户使用WARN日志,加速启动 - ObSimpleClusterTestBase(const std::string &env_prefix = "run_"); + ObSimpleClusterTestBase(const std::string &env_prefix = "run_", + const char *log_disk_size = "10G", + const char *memory_limit = "10G"); virtual ~ObSimpleClusterTestBase(); int start(); @@ -38,7 +40,9 @@ public: observer::ObServer& get_curr_observer() { return cluster_->get_observer(); } observer::ObSimpleServer& get_curr_simple_server() { return *cluster_; } - int create_tenant(const char *tenant_name = "tt1"); + int create_tenant(const char *tenant_name = "tt1", + const char *memory_size = "2G", + const char *log_disk_size = "2G"); int delete_tenant(const char *tenant_name = "tt1"); int get_tenant_id(uint64_t &tenant_id, const char *tenant_name = "tt1"); int exec_write_sql_sys(const char *sql_str, int64_t &affected_rows); diff --git a/mittest/simple_server/env/ob_simple_server.cpp b/mittest/simple_server/env/ob_simple_server.cpp index f7435ac29..779735c23 100644 --- a/mittest/simple_server/env/ob_simple_server.cpp +++ b/mittest/simple_server/env/ob_simple_server.cpp @@ -76,9 +76,13 @@ uint32_t get_local_addr(const char *dev_name) } ObSimpleServer::ObSimpleServer(const std::string &env_prefix, + const char *log_disk_size, + const char *memory_limit, ObServer &server, const std::string &dir_prefix) : server_(server), + log_disk_size_(log_disk_size), + memory_limit_(memory_limit), data_dir_(dir_prefix), run_dir_(env_prefix) { @@ -130,7 +134,9 @@ int ObSimpleServer::simple_init() rs_list_ = local_ip_ + ":" + std::to_string(opts.rpc_port_) + ":" + std::to_string(opts.mysql_port_); opts.rs_list_ = rs_list_.c_str(); // NOTE: memory_limit must keep same with log_disk_size - opts.optstr_ = "log_disk_size=10G,memory_limit=10G,cache_wash_threshold=1G,net_thread_count=4,cpu_count=16,schema_history_expire_time=1d,workers_per_cpu_quota=10,datafile_disk_percentage=2,__min_full_resource_pool_memory=1073741824,system_memory=5G,trace_log_slow_query_watermark=100ms,datafile_size=10G,stack_size=512K"; + optstr_ = std::string(); + optstr_ = optstr_ + "log_disk_size=" + std::string(log_disk_size_) + ",memory_limit=" + std::string(memory_limit_) + ",cache_wash_threshold=1G,net_thread_count=4,cpu_count=16,schema_history_expire_time=1d,workers_per_cpu_quota=10,datafile_disk_percentage=2,__min_full_resource_pool_memory=1073741824,system_memory=5G,trace_log_slow_query_watermark=100ms,datafile_size=10G,stack_size=512K"; + opts.optstr_ = optstr_.c_str(); //opts.devname_ = "eth0"; opts.use_ipv6_ = false; diff --git a/mittest/simple_server/env/ob_simple_server.h b/mittest/simple_server/env/ob_simple_server.h index 2ea4f9571..91008c868 100644 --- a/mittest/simple_server/env/ob_simple_server.h +++ b/mittest/simple_server/env/ob_simple_server.h @@ -30,7 +30,11 @@ public: static const int64_t MAX_WAIT_TENANT_SCHEMA_TIME = 20_s; public: - ObSimpleServer(const std::string &env_prefix, ObServer &server = ObServer::get_instance(), const std::string &dir_prefix = "./store_"); + ObSimpleServer(const std::string &env_prefix, + const char *log_disk_size = "10G", + const char *memory_limit = "10G", + ObServer &server = ObServer::get_instance(), + const std::string &dir_prefix = "./store_"); ~ObSimpleServer() { reset(); } ObServer& get_observer() { return server_; } int simple_init(); @@ -59,8 +63,11 @@ private: std::string local_ip_; int rpc_port_; int mysql_port_; + const char *log_disk_size_; + const char *memory_limit_; std::string data_dir_; std::string rs_list_; + std::string optstr_; std::string run_dir_; common::sqlclient::ObSingleMySQLConnectionPool sql_conn_pool_; common::ObMySQLProxy sql_proxy_; diff --git a/mittest/simple_server/env/ob_simple_server_restart_helper.cpp b/mittest/simple_server/env/ob_simple_server_restart_helper.cpp index 0338da688..adc9f09d1 100644 --- a/mittest/simple_server/env/ob_simple_server_restart_helper.cpp +++ b/mittest/simple_server/env/ob_simple_server_restart_helper.cpp @@ -50,6 +50,8 @@ int ObSimpleServerRestartHelper::run() char log_file_name[name_len]; char rs_log_file_name[name_len]; char election_log_file_name[name_len]; + char trace_log_file_name[name_len]; + char audit_log_file_name[name_len]; char filter_str[name_len]; memset(log_file_name, 0, name_len); memset(rs_log_file_name, 0, name_len); @@ -68,9 +70,12 @@ int ObSimpleServerRestartHelper::run() sprintf(log_file_name, "%s_born.log", test_file_name_); sprintf(rs_log_file_name, "%s_born_rs.log", test_file_name_); sprintf(election_log_file_name, "%s_born_election.log", test_file_name_); + sprintf(trace_log_file_name, "%s_born_trace.log", test_file_name_); + sprintf(audit_log_file_name, "%s_born_audit.log", test_file_name_); sprintf(filter_str, "%s*", born_case_name_); - OB_LOGGER.set_file_name(log_file_name, true, false, rs_log_file_name, election_log_file_name); + OB_LOGGER.set_file_name( + log_file_name, true, false, rs_log_file_name, election_log_file_name, trace_log_file_name, audit_log_file_name); OB_LOGGER.set_log_level(log_level_); ::testing::InitGoogleTest(&argc_, argv_); @@ -84,7 +89,7 @@ int ObSimpleServerRestartHelper::run() int status = 0; wait(&status); if (0 != status) { - fprintf(stdout, "Child process exit with error code\n"); + fprintf(stdout, "Child process exit with error code : %d\n", status); ret = status; return ret; } else { @@ -96,9 +101,12 @@ int ObSimpleServerRestartHelper::run() sprintf(log_file_name, "%s_restart.log", test_file_name_); sprintf(rs_log_file_name, "%s_restart_rs.log", test_file_name_); sprintf(election_log_file_name, "%s_restart_election.log", test_file_name_); + sprintf(trace_log_file_name, "%s_restart_trace.log", test_file_name_); + sprintf(audit_log_file_name, "%s_restart_audit.log", test_file_name_); sprintf(filter_str, "%s*", restart_case_name_); - OB_LOGGER.set_file_name(log_file_name, true, false, rs_log_file_name, election_log_file_name); + OB_LOGGER.set_file_name( + log_file_name, true, false, rs_log_file_name, election_log_file_name, trace_log_file_name, audit_log_file_name); OB_LOGGER.set_log_level(log_level_); ::testing::InitGoogleTest(&argc_, argv_); diff --git a/mittest/simple_server/fast_commit_report.cpp b/mittest/simple_server/fast_commit_report.cpp new file mode 100644 index 000000000..ebaba1f42 --- /dev/null +++ b/mittest/simple_server/fast_commit_report.cpp @@ -0,0 +1,651 @@ +/** + * 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 protected public +#define private public + +#include "env/ob_simple_cluster_test_base.h" +#include "storage/ls/ob_ls.h" +#include "storage/tx_storage/ob_ls_handle.h" +#include "storage/tx_storage/ob_ls_service.h" + +static const char *TEST_FILE_NAME = "fast_commit_report"; +const int64_t TOTAL_FC_ROW_COUNT = 1000000; +const int64_t TOTAL_FC_SESSION = 10; +const int64_t PAINTING_FC_ROW_COUNT = 10000; + +namespace oceanbase +{ + +namespace memtable +{ +int ObMvccValueIterator::get_next_node(const void *&tnode) +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + TRANS_LOG(WARN, "not init", KP(this)); + ret = OB_NOT_INIT; + } else { + tnode = NULL; + while (OB_SUCC(ret) && (NULL == tnode)) { + bool is_lock_node = false; + if (NULL == version_iter_) { + ret = OB_ITER_END; + } else if (OB_FAIL(version_iter_->is_lock_node(is_lock_node))) { + TRANS_LOG(WARN, "fail to check is lock node", K(ret), K(*version_iter_)); + } else if (!(version_iter_->is_aborted() // skip abort version + || is_lock_node + || (NDT_COMPACT == version_iter_->type_ + && skip_compact_))) { + tnode = static_cast(version_iter_); + } + + move_to_next_node_(); + } + } + + return ret; +} + +} + +namespace storage +{ +int LockForReadFunctor::operator()(const ObTxData &tx_data, ObTxCCCtx *tx_cc_ctx) +{ + int ret = OB_ERR_SHARED_LOCK_CONFLICT; + const int64_t MAX_SLEEP_US = 1000; + auto &acc_ctx = lock_for_read_arg_.mvcc_acc_ctx_; + auto lock_expire_ts = acc_ctx.eval_lock_expire_ts(); + + const int32_t state = ATOMIC_LOAD(&tx_data.state_); + + if (OB_ISNULL(tx_cc_ctx) && (ObTxData::RUNNING == state)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "lock for read functor need prepare version.", KR(ret)); + } else { + for (int32_t i = 0; OB_ERR_SHARED_LOCK_CONFLICT == ret; i++) { + if (OB_FAIL(inner_lock_for_read(tx_data, tx_cc_ctx))) { + if (OB_UNLIKELY(observer::SS_STOPPING == GCTX.status_) || + OB_UNLIKELY(observer::SS_STOPPED == GCTX.status_)) { + // rewrite ret + ret = OB_SERVER_IS_STOPPING; + TRANS_LOG(WARN, "observer is stopped", K(ret)); + } else if (ObTimeUtility::current_time() + MIN(i, MAX_SLEEP_US) >= lock_expire_ts) { + ret = OB_ERR_SHARED_LOCK_CONFLICT; + break; + } else if (i < 10) { + PAUSE(); + } else { + ob_usleep((i < MAX_SLEEP_US ? i : MAX_SLEEP_US)); + } + } + } + } + + return ret; +} + +} + +namespace unittest +{ + +using namespace oceanbase::transaction; +using namespace oceanbase::storage; +using namespace oceanbase::memtable; +using namespace oceanbase::storage::checkpoint; + +enum FastCommitTestMode : int +{ + NORMAL_TEST = 0, + BIANQUE_TEST = 1 +}; + +enum FastCommitReportDataMode : int +{ + ALL_CLEANOUT = 0, + ALL_SAME_TXN_DELAY_CLEANOUT = 1, + ALL_DIFF_TXN_DELAY_CLEANOUT = 2 +}; + +enum FastCommitReportTxTableMode : int +{ + ALL_IN_MEMORY = 0, + ALL_IN_DISK = 1 +}; + +FastCommitTestMode fast_commit_test_mode = FastCommitTestMode::NORMAL_TEST; +FastCommitReportDataMode fast_commit_data_mode = FastCommitReportDataMode::ALL_CLEANOUT; +FastCommitReportTxTableMode fast_commit_tx_table_mode = FastCommitReportTxTableMode::ALL_IN_MEMORY; +int64_t total_fc_row_count = TOTAL_FC_ROW_COUNT; +int64_t total_fc_session = TOTAL_FC_SESSION; + +#define EXE_SQL(sql_str) \ + ASSERT_EQ(OB_SUCCESS, sql.assign(sql_str)); \ + ASSERT_EQ(OB_SUCCESS, sql_proxy.write(sql.ptr(), affected_rows)); + +#define EXE_SQL_FMT(...) \ + ASSERT_EQ(OB_SUCCESS, sql.assign_fmt(__VA_ARGS__)); \ + ASSERT_EQ(OB_SUCCESS, sql_proxy.write(sql.ptr(), affected_rows)); + +#define WRITE_SQL_BY_CONN(conn, sql_str) \ + ASSERT_EQ(OB_SUCCESS, sql.assign(sql_str)); \ + ASSERT_EQ(OB_SUCCESS, conn->execute_write(OB_SYS_TENANT_ID, sql.ptr(), affected_rows)); + +#define WRITE_SQL_FMT_BY_CONN(conn, ...) \ + ASSERT_EQ(OB_SUCCESS, sql.assign_fmt(__VA_ARGS__)); \ + ASSERT_EQ(OB_SUCCESS, conn->execute_write(OB_SYS_TENANT_ID, sql.ptr(), affected_rows)); + +#define READ_SQL_BY_CONN(conn, sql_str) \ + ASSERT_EQ(OB_SUCCESS, sql.assign(sql_str)); \ + ASSERT_EQ(OB_SUCCESS, conn->execute_read(OB_SYS_TENANT_ID, sql.ptr(), read_res)); + +class ObFastCommitReport : public ObSimpleClusterTestBase +{ +public: + ObFastCommitReport() : ObSimpleClusterTestBase(TEST_FILE_NAME, "200G", "40G") {} + void prepare_tenant_env() + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + int64_t affected_rows = 0; + ObSqlString sql; + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + WRITE_SQL_BY_CONN(connection, "set GLOBAL ob_trx_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set GLOBAL ob_trx_idle_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set GLOBAL ob_query_timeout = 10000000000"); + } + + void set_private_buffer_size(const char* size) + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL_FMT("alter system set _private_buffer_size = '%s';", size); + } + + void set_fast_commit_count(const int64_t count) + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL_FMT("alter system set _fast_commit_callback_count = %ld;", count); + } + + void set_memstore_limit() + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL("alter system set memstore_limit_percentage = 80;"); + } + + void set_freeze_trigger() + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL("alter system set writing_throttling_trigger_percentage = 100;"); + EXE_SQL("alter system set freeze_trigger_percentage = 80;"); + } + + void create_test_tenant(uint64_t &tenant_id) + { + TRANS_LOG(INFO, "create_tenant start"); + ASSERT_EQ(OB_SUCCESS, create_tenant("tt1", "20G", "100G")); + ASSERT_EQ(OB_SUCCESS, get_tenant_id(tenant_id)); + ASSERT_EQ(OB_SUCCESS, get_curr_simple_server().init_sql_proxy2()); + TRANS_LOG(INFO, "create_tenant end", K(tenant_id)); + } + + // you should use single partition when using it + void get_tablet_id_with_table_name(const char *name, + ObTabletID &tablet) + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + int64_t tablet_id = 0; + + ASSERT_EQ(OB_SUCCESS, sql.assign_fmt("select tablet_id from oceanbase.__all_virtual_table where table_name=%s", name)); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + ASSERT_EQ(OB_SUCCESS, sql_proxy.read(res, sql.ptr())); + sqlclient::ObMySQLResult *result = res.get_result(); + ASSERT_NE(nullptr, result); + ASSERT_EQ(OB_SUCCESS, result->next()); + ASSERT_EQ(OB_SUCCESS, result->get_int("tablet_id", tablet_id)); + } + + tablet = (uint64_t)tablet_id; + } + + void check_no_minor_freeze() + { + TRANS_LOG(INFO, "check no minor freeze start"); + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + int64_t freeze_cnt = 0; + + ASSERT_EQ(OB_SUCCESS, sql.assign("select freeze_cnt from oceanbase.__all_virtual_tenant_memstore_info where tenant_id=1002")); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + ASSERT_EQ(OB_SUCCESS, sql_proxy.read(res, sql.ptr())); + sqlclient::ObMySQLResult *result = res.get_result(); + ASSERT_NE(nullptr, result); + ASSERT_EQ(OB_SUCCESS, result->next()); + ASSERT_EQ(OB_SUCCESS, result->get_int("freeze_cnt", freeze_cnt)); + } + + ASSERT_EQ(0, freeze_cnt); + + TRANS_LOG(INFO, "check no minor freeze END"); + } + + void get_ls(uint64_t tenant_id, ObLS *&ls) + { + ls = nullptr; + share::ObTenantSwitchGuard tenant_guard; + ASSERT_EQ(OB_SUCCESS, tenant_guard.switch_to(tenant_id)); + + ObLSService *ls_svr = MTL(ObLSService*); + ASSERT_NE(nullptr, ls_svr); + ObLSHandle handle; + share::ObLSID ls_id(1001); + ASSERT_EQ(OB_SUCCESS, ls_svr->get_ls(ls_id, handle, ObLSGetMod::STORAGE_MOD)); + ASSERT_NE(nullptr, ls = handle.get_ls()); + } + + void get_memtable(const ObTabletID tablet_id, + ObTableHandleV2 &handle) + { + ObLS *ls = NULL; + get_ls(1002, ls); + ObTabletHandle tablet_handle; + ObTablet *tablet = nullptr; + ASSERT_EQ(OB_SUCCESS, ls->get_tablet_svr()->get_tablet(tablet_id, tablet_handle)); + tablet = tablet_handle.get_obj(); + ASSERT_EQ(OB_SUCCESS, tablet->get_active_memtable(handle)); + } + + void insert_data_single(const int row_count = oceanbase::unittest::total_fc_row_count) + { + ObSqlString sql; + int64_t affected_rows = 0; + + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + + WRITE_SQL_BY_CONN(connection, "set SESSION ob_trx_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set SESSION ob_trx_idle_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set SESSION ob_query_timeout = 10000000000"); + + TRANS_LOG(INFO, "insert data start"); + const int64_t begin_time = ObTimeUtility::current_time(); + + if (FastCommitReportDataMode::ALL_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + WRITE_SQL_BY_CONN(connection, "begin;"); + } else if (FastCommitReportDataMode::ALL_SAME_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + WRITE_SQL_BY_CONN(connection, "begin;"); + } else if (FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + // pass + } + for (int i = 0; i < row_count; i++) { + if (FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + WRITE_SQL_BY_CONN(connection, "begin;"); + } + // const int64_t single_begin_time = ObTimeUtility::current_time(); + WRITE_SQL_FMT_BY_CONN(connection, "insert into test_fast_commit values(1);"); + if (FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + WRITE_SQL_BY_CONN(connection, "commit;"); + } + // const int64_t single_end_time = ObTimeUtility::current_time(); + // TRANS_LOG(INFO, "single insert data single cost", K(single_end_time - single_begin_time)); + if ((i + 1) % PAINTING_FC_ROW_COUNT == 0) { + TRANS_LOG(INFO, "insert data single pass one round", + K(oceanbase::unittest::fast_commit_data_mode), + K(oceanbase::unittest::fast_commit_tx_table_mode), + K(i + 1)); + } + } + if (FastCommitReportDataMode::ALL_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + WRITE_SQL_BY_CONN(connection, "commit;"); + } else if (FastCommitReportDataMode::ALL_SAME_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + WRITE_SQL_BY_CONN(connection, "commit;"); + } else if (FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + // pass + } + const int64_t end_time = ObTimeUtility::current_time(); + + TRANS_LOG(INFO, "insert data single cost", K(end_time - begin_time), K(begin_time), K(end_time)); + } + + void insert_data_parallel(const int parrallel_num = oceanbase::unittest::total_fc_session, + const int row_count = oceanbase::unittest::total_fc_row_count) + { + int single_row_count = row_count / parrallel_num; + std::thread *threads[parrallel_num]; + const int64_t begin_time = ObTimeUtility::current_time(); + + for (int i = 0; i < parrallel_num; i++) { + threads[i] = new std::thread(&ObFastCommitReport::insert_data_single, this, single_row_count); + } + + for (int i = 0; i < parrallel_num; i++) { + threads[i]->join(); + } + const int64_t end_time = ObTimeUtility::current_time(); + TRANS_LOG(INFO, "insert data parallel cost", K(end_time - begin_time), K(begin_time), K(end_time)); + } + + void read_data(const int row_count = oceanbase::unittest::total_fc_row_count) + { + ObSqlString sql; + int64_t affected_rows = 0; + int64_t cnt = 0; + int ret = 0; + ObISQLClient::ReadResult read_res; + + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + + WRITE_SQL_BY_CONN(connection, "set SESSION ob_trx_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set SESSION ob_trx_idle_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set SESSION ob_query_timeout = 10000000000"); + + ObLS *ls = NULL; + get_ls(1002, ls); + ObTxTable *tx_table = ls->get_tx_table(); + + const int64_t begin_time = ObTimeUtility::current_time(); + READ_SQL_BY_CONN(connection, "select count(*) as cnt from test_fast_commit"); + const int64_t end_time = ObTimeUtility::current_time(); + TRANS_LOG(INFO, "read data cost", K(end_time - begin_time), K(begin_time), K(end_time)); + std::cout << "read data cost(total_cost=" << (end_time - begin_time) + << ", begin_time=" << begin_time + << ", end_time=" << end_time << ")\n"; + + sqlclient::ObMySQLResult *result = read_res.get_result(); + ASSERT_NE(nullptr, result); + ASSERT_EQ(OB_SUCCESS, result->next()); + ASSERT_EQ(OB_SUCCESS, result->get_int("cnt", cnt)); + + ASSERT_EQ(cnt, row_count); + } + + void check_memtable_cleanout(const bool memtable_is_all_cleanout, + const bool memtable_is_all_delay_cleanout, + const int64_t memtable_count) + { + ObSqlString sql; + int64_t affected_rows = 0; + ObTableHandleV2 handle; + get_memtable(ObTabletID(200001), handle); + + ObIMemtable *imemtable; + handle.get_memtable(imemtable); + ObMemtable *memtable = dynamic_cast(imemtable); + + bool is_all_cleanout = true; + bool is_all_delay_cleanout = true; + int64_t count = 0; + + memtable->dump2text("/tmp/fast_commit_report/memtable.txt"); + ASSERT_EQ(OB_SUCCESS, memtable->check_cleanout(is_all_cleanout, + is_all_delay_cleanout, + count)); + + ASSERT_EQ(memtable_is_all_cleanout, is_all_cleanout); + ASSERT_EQ(memtable_is_all_delay_cleanout, is_all_delay_cleanout); + ASSERT_EQ(memtable_count, count); + } + + void wait_freeze_tx_table_finish(const share::SCN start_log_ts, + const share::SCN end_log_ts) + { + bool ok = false; + TRANS_LOG(INFO, "check tx table minor freeze finish start"); + while (!ok) { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + ObString state; + + ASSERT_EQ(OB_SUCCESS, sql.assign_fmt("select state from oceanbase.__all_virtual_tx_data_table where tenant_id=1002 and ls_id = 1001 and start_scn = %ld and end_scn = %ld", start_log_ts.get_val_for_tx(), end_log_ts.get_val_for_tx())); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + ASSERT_EQ(OB_SUCCESS, sql_proxy.read(res, sql.ptr())); + sqlclient::ObMySQLResult *result = res.get_result(); + ASSERT_NE(nullptr, result); + ASSERT_EQ(OB_SUCCESS, result->next()); + ASSERT_EQ(OB_SUCCESS, result->get_varchar("state", state)); + } + + if (state == "MINI") { + ok = true; + } else { + usleep(1 * 1000 * 1000); + } + } + + usleep(5 * 1000 * 1000); + + TRANS_LOG(INFO, "check tx table minor freeze finish end"); + } + + void minor_freeze_tx_data_memtable() + { + TRANS_LOG(INFO, "minor_freeze_tx_data_memtable begin"); + + ObLS *ls = NULL; + ObTxDataMemtable *tx_data_memtable = NULL; + ObMemtableMgrHandle memtable_mgr_handle; + ObIMemtableMgr *memtable_mgr = nullptr; + ObTableHandleV2 handle; + + get_ls(1002, ls); + + ASSERT_EQ(OB_SUCCESS, ls->get_tablet_svr()->get_tx_data_memtable_mgr(memtable_mgr_handle)); + memtable_mgr = memtable_mgr_handle.get_memtable_mgr(); + ASSERT_EQ(OB_SUCCESS, memtable_mgr->get_active_memtable(handle)); + ASSERT_EQ(OB_SUCCESS, handle.get_tx_data_memtable(tx_data_memtable)); + + tx_data_memtable->dump2text("/tmp/fast_commit_report/tx_data_memtable.txt"); + ASSERT_EQ(OB_SUCCESS, ((ObTxDataMemtableMgr *)(memtable_mgr))->freeze()); + share::SCN start_log_ts = tx_data_memtable->get_start_scn(); + share::SCN end_log_ts = tx_data_memtable->get_end_scn(); + ASSERT_EQ(OB_SUCCESS, tx_data_memtable->flush()); + + wait_freeze_tx_table_finish(start_log_ts, end_log_ts); + + TRANS_LOG(INFO, "minor_freeze_tx_data_memtable end"); + } +private: + +}; + +TEST_F(ObFastCommitReport, fast_commit_report) +{ + ObSqlString sql; + int64_t affected_rows = 0; + + // ============================== Phase1. create tenant and table ============================== + TRANS_LOG(INFO, "create tenant start"); + uint64_t tenant_id = 0; + create_test_tenant(tenant_id); + TRANS_LOG(INFO, "create tenant end"); + + share::ObTenantSwitchGuard tenant_guard; + ASSERT_EQ(OB_SUCCESS, tenant_guard.switch_to(tenant_id)); + + TRANS_LOG(INFO, "create table start"); + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + EXE_SQL("create table test_fast_commit (a int)"); + usleep(10 * 1000 * 1000); + TRANS_LOG(INFO, "create_table end"); + + prepare_tenant_env(); + set_freeze_trigger(); + set_private_buffer_size("2M"); + + if (FastCommitReportDataMode::ALL_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + set_fast_commit_count(oceanbase::unittest::total_fc_row_count + 100); + } else if (FastCommitReportDataMode::ALL_SAME_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + set_fast_commit_count(0); + } else if (FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + set_fast_commit_count(0); + } + + if (FastCommitReportDataMode::ALL_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + insert_data_parallel(); + } else if (FastCommitReportDataMode::ALL_SAME_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + insert_data_single(); + } else if (FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + insert_data_parallel(); + } + + check_no_minor_freeze(); + + if (FastCommitReportDataMode::ALL_CLEANOUT == oceanbase::unittest::fast_commit_data_mode) { + check_memtable_cleanout(true, /*is_all_cleanout*/ + false, /*is_all_delay_cleanout*/ + oceanbase::unittest::total_fc_row_count); + } else { + check_memtable_cleanout(false, /*is_all_cleanout*/ + true, /*is_all_delay_cleanout*/ + oceanbase::unittest::total_fc_row_count); + } + + if (FastCommitReportTxTableMode::ALL_IN_DISK == oceanbase::unittest::fast_commit_tx_table_mode) { + minor_freeze_tx_data_memtable(); + } + + TRANS_LOG(INFO, "fast commit report with arguments", + K(oceanbase::unittest::fast_commit_data_mode), + K(oceanbase::unittest::fast_commit_tx_table_mode)); + + if (oceanbase::unittest::FastCommitTestMode::NORMAL_TEST == oceanbase::unittest::fast_commit_test_mode) { + // check with 3 times + read_data(); + read_data(); + read_data(); + } else if (oceanbase::unittest::FastCommitTestMode::BIANQUE_TEST == oceanbase::unittest::fast_commit_test_mode) { + std::cout << "Master, you should start enterring a number as you like\n"; + int n; + std::cin >> n; + std::cout << "Your faithful servant, qianchen is start reading data within 60 seconds\n"; + std::cout << "So prepare for command: sudo perf record -e cycles -c 100000000 -p $(pidof -s test_fast_commit_report) -g -- sleep 30\n"; + int64_t base_ts = OB_TSC_TIMESTAMP.current_time(); + + while (OB_TSC_TIMESTAMP.current_time() - base_ts <= 60 * 1000 * 1000) { + read_data(); + } + + std::cout << "Data reading has been finished, you can generate it with command: sudo perf script -i perf.data -F ip,sym -f > data.viz\n"; + std::cout << "And finally you can graph it using url: http://tools.obdev.alibaba-inc.com:8888/perf/ \n"; + } +} + + +} // namespace unittest +} // namespace oceanbase + +void tutorial() +{ + std::cout << "./mittest/simple_server/test_fast_commit_report -m $1 -d $2 -t $3 -r $4 -s $5\n" + << "-m(mode): 0 = NORMAL_MODE; 1 = BIANQUE_MODE\n" + << "-d(data model): 0 = ALL_CLEANOUT; 1 = ALL_SAME_TXN_DELAY_CLEANOUT; 2 = ALL_DIFF_TXN_DELAY_CLEANOUT\n" + << "-t(tx data model): 0 = ALL_IN_MEMORY; 1 = ALL_IN_DISK\n" + << "-r(row count): n = n row that is read during benchmark\n" + << "-s(session count): n = n session that is used during insert before benchmark\n"; +} + +int main(int argc, char **argv) +{ + int c = 0; + while(EOF != (c = getopt(argc,argv,"h:m:d:t:r:s:"))) { + switch(c) { + case 'h': + tutorial(); + return 0; + case 'm': + fprintf(stdout, "m : %s\n", optarg); + oceanbase::unittest::fast_commit_test_mode = (oceanbase::unittest::FastCommitTestMode)atoi(optarg); + break; + case 'd': + oceanbase::unittest::fast_commit_data_mode = (oceanbase::unittest::FastCommitReportDataMode)atoi(optarg); + break; + case 't': + oceanbase::unittest::fast_commit_tx_table_mode = (oceanbase::unittest::FastCommitReportTxTableMode)atoi(optarg); + break; + case 'r': + oceanbase::unittest::total_fc_row_count = (int64_t)atoi(optarg); + break; + case 's': + oceanbase::unittest::total_fc_session = (int64_t)atoi(optarg); + break; + default: + break; + } + } + + if (oceanbase::unittest::fast_commit_test_mode != oceanbase::unittest::FastCommitTestMode::NORMAL_TEST + && oceanbase::unittest::fast_commit_test_mode != oceanbase::unittest::FastCommitTestMode::BIANQUE_TEST) { + TRANS_LOG(ERROR, "wrong choice", K(oceanbase::unittest::fast_commit_test_mode)); + ob_abort(); + } + + if (oceanbase::unittest::fast_commit_data_mode != oceanbase::unittest::FastCommitReportDataMode::ALL_CLEANOUT + && oceanbase::unittest::fast_commit_data_mode != oceanbase::unittest::FastCommitReportDataMode::ALL_SAME_TXN_DELAY_CLEANOUT + && oceanbase::unittest::fast_commit_data_mode != oceanbase::unittest::FastCommitReportDataMode::ALL_DIFF_TXN_DELAY_CLEANOUT) { + TRANS_LOG(ERROR, "wrong choice", K(oceanbase::unittest::fast_commit_data_mode)); + ob_abort(); + } + + if (oceanbase::unittest::fast_commit_tx_table_mode != oceanbase::unittest::FastCommitReportTxTableMode::ALL_IN_MEMORY + && oceanbase::unittest::fast_commit_tx_table_mode != oceanbase::unittest::FastCommitReportTxTableMode::ALL_IN_DISK) { + TRANS_LOG(ERROR, "wrong choice", K(oceanbase::unittest::fast_commit_tx_table_mode)); + ob_abort(); + } + + if (oceanbase::unittest::total_fc_row_count < oceanbase::unittest::total_fc_session) { + TRANS_LOG(ERROR, "wrong choice", K(oceanbase::unittest::total_fc_row_count), K(oceanbase::unittest::total_fc_session)); + ob_abort(); + } + + oceanbase::unittest::init_log_and_gtest(argc, argv); + OB_LOGGER.set_log_level("info"); + ::testing::InitGoogleTest(&argc, argv); + + TRANS_LOG(INFO, "fast commit report with arguments", + K(oceanbase::unittest::fast_commit_test_mode), + K(oceanbase::unittest::fast_commit_data_mode), + K(oceanbase::unittest::fast_commit_tx_table_mode), + K(oceanbase::unittest::total_fc_row_count), + K(oceanbase::unittest::total_fc_session)); + + return RUN_ALL_TESTS(); +} diff --git a/mittest/simple_server/rewrite_function_for_test_big_tx_data.cpp b/mittest/simple_server/rewrite_function_for_test_big_tx_data.cpp new file mode 100644 index 000000000..f4fe56c52 --- /dev/null +++ b/mittest/simple_server/rewrite_function_for_test_big_tx_data.cpp @@ -0,0 +1,320 @@ +#include "storage/tx_table/ob_tx_table_iterator.h" +#include "storage/tx/ob_tx_data_define.h" +#include "storage/tx/ob_trans_part_ctx.h" +#include "share/ob_table_access_helper.h" +#include "lib/ob_define.h" +#include "observer/ob_server_struct.h" +#include "share/rc/ob_tenant_base.h" +#include "storage/ls/ob_ls_meta.h" +#include "storage/ls/ob_ls.h" +#include "storage/tx_storage/ob_ls_service.h" +#include "env/ob_simple_cluster_test_base.h" +#include "lib/mysqlclient/ob_mysql_result.h" +#include "storage/tx_storage/ob_ls_handle.h" +#include "storage/compaction/ob_partition_parallel_merge_ctx.h" +#include "storage/compaction/ob_tablet_merge_ctx.h" +#include +int64_t TEST_TX_ID = 0; +bool DUMP_BIG_TX_DATA = false; +bool LOAD_BIG_TX_DATA = false; +bool READ_TEST_TX_FROM_SSTABLE = false; +int64_t BIGGEST_TX_DATA_SIZE = 0; + +namespace oceanbase +{ +using namespace common; +using namespace share; +using namespace transaction; +namespace storage +{ +int ObTxData::add_undo_action(ObTxTable *tx_table, + transaction::ObUndoAction &new_undo_action, + ObUndoStatusNode *undo_node) +{ + // STORAGE_LOG(DEBUG, "do add_undo_action"); + UNUSED(undo_node); + int ret = OB_SUCCESS; + SpinWLockGuard guard(undo_status_list_.lock_); + ObTxDataTable *tx_data_table = nullptr; + ObUndoStatusNode *node = undo_status_list_.head_; + if (OB_ISNULL(tx_table)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "tx table is nullptr.", KR(ret)); + } 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 { +/**************************************************************************************************/ + // 在测试big tx data的时候,连续挂很多无效的undo action上去,并且disable掉merge的逻辑 + // 可以节省测试时间,否则构建big tx data的耗时太长,没办法加到farm里 + int loop_times = 10000; + while (OB_SUCC(ret) && --loop_times) { + ObUndoStatusNode *new_node = nullptr; + if (OB_FAIL(tx_data_table->alloc_undo_status_node(new_node))) { + STORAGE_LOG(WARN, "alloc_undo_status_node() fail", KR(ret)); + } else { + new_node->next_ = node; + undo_status_list_.head_ = new_node; + node = new_node; + undo_status_list_.undo_node_cnt_++; + } + for (int64_t idx = 0; idx < TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE; ++idx) { + node->undo_actions_[node->size_++] = new_undo_action; + } + } +/**************************************************************************************************/ + } + return ret; +} + +int ObTxDataMemtableScanIterator + ::TxData2DatumRowConverter::generate_next_now(const blocksstable::ObDatumRow *&row) +{ + const int64_t SSTABLE_HIDDEN_COLUMN_CNT = ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt(); + int ret = OB_SUCCESS; + if (OB_ISNULL(tx_data_)) { + ret = OB_ITER_END;// no tx data remained + /*****************[NEED REMOVE IN FUTURE]*****************/ + } else if (INT64_MAX == tx_data_->tx_id_.get_id() && + generate_size_ == 1) { + ret = OB_ITER_END;// fake tx data datum row has been generated + /*********************************************************/ + } else if (INT64_MAX != tx_data_->tx_id_.get_id() && + generate_size_ == std::ceil(buffer_len_ * 1.0 / common::OB_MAX_VARCHAR_LENGTH)) { + ret = OB_ITER_END;// all tx data datum row has been generated + } else { + if (generate_size_ >= 1) { + STORAGE_LOG(INFO, "meet big tx data", KR(ret), K(*this)); +/**************************************************************************************************/ + if (buffer_len_ > ATOMIC_LOAD(&BIGGEST_TX_DATA_SIZE)) { + // not exactly accurate, but enough for unittest + ATOMIC_STORE(&BIGGEST_TX_DATA_SIZE, buffer_len_); + } + if (tx_data_->undo_status_list_.undo_node_cnt_ > 0) { + std::cout << "tx_id:" << tx_data_->tx_id_.get_id() << ", undo cnt:" << tx_data_->undo_status_list_.undo_node_cnt_ << ", generate size:" << generate_size_ << std::endl; + } + ATOMIC_STORE(&DUMP_BIG_TX_DATA, true); +/**************************************************************************************************/ + } + datum_row_.reset(); + new (&datum_row_) ObDatumRow();// CAUTIONS: this is needed, or will core dump + if (OB_FAIL(datum_row_.init(DEFAULT_TX_DATA_ALLOCATOR, + TX_DATA_MAX_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT))) { + STORAGE_LOG(ERROR, "fail to init datum row", KR(ret), K(*this)); + } else { + datum_row_.row_flag_.set_flag(blocksstable::ObDmlFlag::DF_INSERT); + datum_row_.storage_datums_[TX_DATA_ID_COLUMN].set_int(tx_data_->tx_id_.get_id()); + datum_row_.storage_datums_[TX_DATA_IDX_COLUMN].set_int(generate_size_); + datum_row_.storage_datums_[TX_DATA_IDX_COLUMN + 1].set_int(-4096);// storage layer needed + datum_row_.storage_datums_[TX_DATA_IDX_COLUMN + 2].set_int(0);// storage layer needed + int64_t total_row_cnt_column = TX_DATA_TOTAL_ROW_CNT_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; + int64_t end_ts_column = TX_DATA_END_TS_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; + int64_t value_column = TX_DATA_VAL_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; + char *p_value_begin = serialize_buffer_ + common::OB_MAX_VARCHAR_LENGTH * generate_size_; + generate_size_++; + ObString value; + /*****************[NEED REMOVE IN FUTURE]*****************/ + // TODO : remove this after the sstables do not need upper trans version + if (INT64_MAX == tx_data_->tx_id_.get_id()) { + // NOTE : this fake tx data is generated in + // ObTxDataMemtable::pre_process_commit_version_row_ + datum_row_.storage_datums_[total_row_cnt_column].set_int(1); + datum_row_.storage_datums_[end_ts_column].set_int(INT64_MAX); + value.assign((char *)tx_data_->start_scn_.get_val_for_tx(), tx_data_->commit_version_.get_val_for_tx()); + /*********************************************************/ + } else { + datum_row_.storage_datums_[total_row_cnt_column].set_int(std::ceil(buffer_len_ * 1.0 / common::OB_MAX_VARCHAR_LENGTH)); + datum_row_.storage_datums_[end_ts_column].set_int(tx_data_->end_scn_.get_val_for_tx()); + value.assign(p_value_begin, + std::min(common::OB_MAX_VARCHAR_LENGTH, + buffer_len_ - (p_value_begin - serialize_buffer_))); + } + datum_row_.storage_datums_[value_column].set_string(value); + datum_row_.set_first_multi_version_row();// storage layer needed for compatibility + datum_row_.set_last_multi_version_row();// storage layer needed for compatibility + datum_row_.set_compacted_multi_version_row();// storage layer needed for compatibility + row = &datum_row_; + } + } + return ret; +} + +int ObTxDataSingleRowGetter::deserialize_tx_data_from_store_buffers_(ObTxData &tx_data) +{ + int ret = OB_SUCCESS; + int64_t total_buffer_size = 0; + for (int64_t idx = 0; idx < tx_data_buffers_.count(); ++idx) { + total_buffer_size += tx_data_buffers_[idx].get_ob_string().length(); + } + OB_ASSERT(total_buffer_size != 0); + char *merge_buffer = (char*)DEFAULT_TX_DATA_ALLOCATOR.alloc(total_buffer_size); + int64_t pos = 0; + if (OB_ISNULL(merge_buffer)) { + STORAGE_LOG(ERROR, "fail to alloc merge buffer", KR(ret), K(total_buffer_size)); + } else { + char *p_dest = merge_buffer; + for (int64_t idx = 0; idx < tx_data_buffers_.count(); ++idx) { + OB_ASSERT(p_dest + tx_data_buffers_[idx].get_ob_string().length() <= merge_buffer + total_buffer_size); + memcpy(p_dest, tx_data_buffers_[idx].get_ob_string().ptr(), tx_data_buffers_[idx].get_ob_string().length()); + p_dest += tx_data_buffers_[idx].get_ob_string().length(); + } + tx_data.tx_id_ = tx_id_; + if (OB_FAIL(tx_data.deserialize(merge_buffer, total_buffer_size, pos, slice_allocator_))) { + STORAGE_LOG(WARN, "deserialize tx data failed", KR(ret), KPHEX(merge_buffer, total_buffer_size)); + hex_dump(merge_buffer, total_buffer_size, true, OB_LOG_LEVEL_WARN); + } else if (!tx_data.is_valid_in_tx_data_table()) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "the deserialized tx data is invalid.", KR(ret), K(tx_data)); + } +/**************************************************************************************************/ + if (tx_data.tx_id_.get_id() == ATOMIC_LOAD(&TEST_TX_ID)) { + if (tx_data_buffers_.count() > 1) { + ATOMIC_STORE(&LOAD_BIG_TX_DATA, true); + std::cout << "read big tx id from sstable, tx_id:" << ATOMIC_LOAD(&TEST_TX_ID) << ", undo cnt:" << tx_data.undo_status_list_.undo_node_cnt_ << ", buffer cnt:" << tx_data_buffers_.count() << std::endl; + } + } +/**************************************************************************************************/ + } + if (OB_NOT_NULL(merge_buffer)) { + DEFAULT_TX_DATA_ALLOCATOR.free(merge_buffer); + } + return ret; +} + +int ObTxDataTable::check_with_tx_data(const ObTransID tx_id, ObITxDataCheckFunctor &fn) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "tx data table is not init.", KR(ret), KP(this), K(tx_id)); + } else if (OB_SUCC(check_tx_data_in_memtable_(tx_id, fn))) { + // successfully do check function in memtable, check done + STORAGE_LOG(DEBUG, "tx data table check with tx memtable data succeed", K(tx_id), K(fn)); + } else if (OB_TRANS_CTX_NOT_EXIST == ret && OB_SUCC(check_tx_data_in_sstable_(tx_id, fn))) { +/**************************************************************************************************/ + if (tx_id.get_id() == ATOMIC_LOAD(&TEST_TX_ID)) { + ATOMIC_STORE(&READ_TEST_TX_FROM_SSTABLE, true); + } +/**************************************************************************************************/ + // successfully do check function in sstable + STORAGE_LOG(DEBUG, "tx data table check with tx sstable data succeed", K(tx_id), K(fn)); + } else { + STORAGE_LOG(WARN, "check something in tx data fail.", KR(ret), K(tx_id), KP(this), + K(tablet_id_)); + } + + return ret; +} + +int ObParallelMergeCtx::init_parallel_mini_merge(compaction::ObTabletMergeCtx &merge_ctx) +{ + int ret = OB_SUCCESS; + + if (OB_UNLIKELY(MINI_MERGE != merge_ctx.param_.merge_type_)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid argument to init parallel mini merge", K(ret), K(merge_ctx)); + } else { + const int64_t tablet_size = merge_ctx.get_schema()->get_tablet_size(); + memtable::ObIMemtable *memtable = nullptr; + if (OB_FAIL(merge_ctx.tables_handle_.get_first_memtable(memtable))) { + STORAGE_LOG(WARN, "failed to get first memtable", K(ret), + "merge tables", merge_ctx.tables_handle_); + } else { + int64_t total_bytes = 0; + int64_t total_rows = 0; + int32_t mini_merge_thread = 0; + if (OB_FAIL(memtable->estimate_phy_size(nullptr, nullptr, total_bytes, total_rows))) { + STORAGE_LOG(WARN, "Failed to get estimate size from memtable", K(ret)); + } else if (MTL(ObTenantDagScheduler *)->get_up_limit(ObDagPrio::DAG_PRIO_COMPACTION_HIGH, mini_merge_thread)) { + STORAGE_LOG(WARN, "failed to get uplimit", K(ret), K(mini_merge_thread)); + } else { + ObArray store_ranges; + mini_merge_thread = MAX(mini_merge_thread, PARALLEL_MERGE_TARGET_TASK_CNT); +/**************************************************************************************************/ + concurrent_cnt_ = 2; + // concurrent_cnt_ = MIN((total_bytes + tablet_size - 1) / tablet_size, mini_merge_thread); +/**************************************************************************************************/ + if (concurrent_cnt_ <= 1) { + if (OB_FAIL(init_serial_merge())) { + STORAGE_LOG(WARN, "Failed to init serialize merge", K(ret)); + } + } else if (OB_FAIL(memtable->get_split_ranges(nullptr, nullptr, concurrent_cnt_, store_ranges))) { + if (OB_ENTRY_NOT_EXIST == ret) { + if (OB_FAIL(init_serial_merge())) { + STORAGE_LOG(WARN, "Failed to init serialize merge", K(ret)); + } + } else { + STORAGE_LOG(WARN, "Failed to get split ranges from memtable", K(ret)); + } + } else if (OB_UNLIKELY(store_ranges.count() != concurrent_cnt_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "Unexpected range array and concurrent_cnt", K(ret), K_(concurrent_cnt), + K(store_ranges)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < store_ranges.count(); i++) { + ObDatumRange datum_range; + if (OB_FAIL(datum_range.from_range(store_ranges.at(i), allocator_))) { + STORAGE_LOG(WARN, "Failed to transfer store range to datum range", K(ret), K(i), K(store_ranges.at(i))); + } else if (OB_FAIL(range_array_.push_back(datum_range))) { + STORAGE_LOG(WARN, "Failed to push back merge range to array", K(ret), K(datum_range)); + } + } + parallel_type_ = PARALLEL_MINI; + STORAGE_LOG(INFO, "Succ to get parallel mini merge ranges", K_(concurrent_cnt), K_(range_array)); + } + } + } + } + + return ret; +} + +int ObParallelMergeCtx::init(compaction::ObTabletMergeCtx &merge_ctx) +{ + int ret = OB_SUCCESS; + + if (IS_INIT) { + ret = OB_INIT_TWICE; + STORAGE_LOG(WARN, "ObParallelMergeCtx init twice", K(ret)); + } else if (OB_UNLIKELY(nullptr == merge_ctx.get_schema() || merge_ctx.tables_handle_.empty())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid argument to init parallel merge", K(ret), K(merge_ctx)); + } else { + int64_t tablet_size = merge_ctx.get_schema()->get_tablet_size(); +/**************************************************************************************************/ + bool enable_parallel_minor_merge = true; + // bool enable_parallel_minor_merge = false; + // { + // omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + // if (tenant_config.is_valid()) { + // enable_parallel_minor_merge = tenant_config->_enable_parallel_minor_merge; + // } + // } +/**************************************************************************************************/ + if (enable_parallel_minor_merge && tablet_size > 0 && is_mini_merge(merge_ctx.param_.merge_type_)) { + if (OB_FAIL(init_parallel_mini_merge(merge_ctx))) { + STORAGE_LOG(WARN, "Failed to init parallel setting for mini merge", K(ret)); + } + } else if (enable_parallel_minor_merge && tablet_size > 0 && is_minor_merge(merge_ctx.param_.merge_type_)) { + if (OB_FAIL(init_parallel_mini_minor_merge(merge_ctx))) { + STORAGE_LOG(WARN, "Failed to init parallel setting for mini minor merge", K(ret)); + } + } else if (tablet_size > 0 && is_major_merge_type(merge_ctx.param_.merge_type_)) { + if (OB_FAIL(init_parallel_major_merge(merge_ctx))) { + STORAGE_LOG(WARN, "Failed to init parallel major merge", K(ret)); + } + } else if (OB_FAIL(init_serial_merge())) { + STORAGE_LOG(WARN, "Failed to init serialize merge", K(ret)); + } + if (OB_SUCC(ret)) { + is_inited_ = true; + STORAGE_LOG(INFO, "Succ to init parallel merge ctx", + K(enable_parallel_minor_merge), K(tablet_size), K(merge_ctx.param_)); + } + } + + return ret; +} + +} +} \ No newline at end of file diff --git a/mittest/simple_server/test_big_tx_data.cpp b/mittest/simple_server/test_big_tx_data.cpp new file mode 100644 index 000000000..d49fc18b5 --- /dev/null +++ b/mittest/simple_server/test_big_tx_data.cpp @@ -0,0 +1,215 @@ +#include +#include +#include +#define protected public +#define private public +#include "share/ob_table_access_helper.h" +#include "lib/ob_define.h" +#include "observer/ob_server_struct.h" +#include "share/rc/ob_tenant_base.h" +#include "storage/ls/ob_ls_meta.h" +#include "storage/ls/ob_ls.h" +#include "storage/tx_storage/ob_ls_service.h" +#include "env/ob_simple_cluster_test_base.h" +#include "lib/mysqlclient/ob_mysql_result.h" +#include "storage/tx_storage/ob_ls_handle.h" +#include + +#include "rewrite_function_for_test_big_tx_data.cpp" + +using namespace std; + +namespace oceanbase +{ +using namespace storage; +namespace unittest +{ + +using namespace oceanbase::transaction; +using namespace oceanbase::storage; + +class TestRunCtx +{ +public: + uint64_t tenant_id_ = 0; + int time_sec_ = 0; +}; + +TestRunCtx RunCtx; + +class TestBigTxData : public ObSimpleClusterTestBase +{ +public: + // 指定case运行目录前缀 test_ob_simple_cluster_ + void minor_freeze_and_wait(); + TestBigTxData() : ObSimpleClusterTestBase("test_big_tx_data_") {} +}; + +#define DO(stmt) /*cout << "before do line:" << __LINE__ << endl;*/ASSERT_EQ((stmt), OB_SUCCESS);/*cout << "after do line:" << __LINE__ << endl;*/ +#define EXEC_SQL(sql) connection->execute_write(OB_SYS_TENANT_ID, sql, affected_rows) + +class DoNothingOP : public ObITxDataCheckFunctor +{ + virtual int operator()(const ObTxData &tx_data, ObTxCCCtx *tx_cc_ctx = nullptr) { + UNUSED(tx_cc_ctx); + cout << "read tx data:" << tx_data.tx_id_.get_id() << ", undo cnt:" << tx_data.undo_status_list_.undo_node_cnt_ << ", READ_TEST_TX_FROM_SSTABLE:" << ATOMIC_LOAD(&READ_TEST_TX_FROM_SSTABLE) << endl; + return OB_SUCCESS; + } +}; + +TEST_F(TestBigTxData, big_tx_data) +{ + int ret = OB_SUCCESS; + MTL_SWITCH(OB_SYS_TENANT_ID) { + int64_t affected_rows = 0; + sqlclient::ObISQLConnection *connection = nullptr; + // 1,开启事务,生成一些savepoint + ASSERT_EQ(OB_SUCCESS, get_curr_simple_server().get_sql_proxy().acquire(connection)); + DO(EXEC_SQL("set ob_trx_timeout = 6000000000")); + DO(EXEC_SQL("set ob_trx_idle_timeout = 6000000000")); + DO(EXEC_SQL("alter system set _fast_commit_callback_count = 0"));// disallow fast commit + DO(EXEC_SQL("create table test_big_tx_data (a int primary key, b int)")); + DO(EXEC_SQL("alter system set _private_buffer_size = '1B'")); + DO(EXEC_SQL("begin")); + constexpr int64_t savepoint_size = 12; + for (int64_t idx = 0; idx < savepoint_size; ++idx) { + ObSqlString sql; + DO(sql.append_fmt("insert into test_big_tx_data values(%ld, 1)", idx)); + DO(EXEC_SQL(sql.ptr())); + sql.reset(); + DO(sql.append_fmt("savepoint s%ld", idx)); + DO(EXEC_SQL(sql.ptr())); + } + // 2,获取本事务ID,这个事务的tx data足够大 + HEAP_VAR(ObMySQLProxy::MySQLResult, res) { + int64_t sess_id = 0; + int64_t tx_id; + DO(connection->execute_read(OB_SYS_TENANT_ID, "select connection_id()", res)); + common::sqlclient::ObMySQLResult *result = res.mysql_result(); + DO(result->next()); + result->get_int("connection_id()", sess_id); + ObSqlString sql; + DO(sql.append_fmt("select trans_id from oceanbase.__all_virtual_processlist where id=%ld", sess_id)); + res.reset(); + DO(connection->execute_read(OB_SYS_TENANT_ID, sql.ptr(), res)); + result = res.mysql_result(); + DO(result->next()); + result->get_int("trans_id", tx_id); + ATOMIC_STORE(&TEST_TX_ID, tx_id); + std::cout << "tx_id:" << tx_id << std::endl; + } + // 3,写日志才可以生成undo status + cout << "alter system minor freeze 1" << endl; + minor_freeze_and_wait(); + // 4,回滚生成undo status + for (int64_t idx = savepoint_size - 1; idx >= 0; --idx) { + ObSqlString sql; + DO(sql.append_fmt("rollback to s%ld", idx)); + DO(EXEC_SQL(sql.ptr())); + } + DO(EXEC_SQL("commit")); + ::sleep(10); + // 5,把tx data转下去 + cout << "alter system minor freeze 2" << endl; + minor_freeze_and_wait(); + ASSERT_EQ(ATOMIC_LOAD(&DUMP_BIG_TX_DATA), true);// 确保tx data已经走过了序列化逻辑 + // 6, 读一次这个tx data + DoNothingOP op; + ObLSService *ls_service = MTL(ObLSService*); + ObLSHandle handle; + DO(ls_service->get_ls(ObLSID(1), handle, storage::ObLSGetMod::DEADLOCK_MOD)); + STORAGETEST_LOG(INFO, "start read tx data from sstable", K(TEST_TX_ID)); + DO(handle.get_ls()->tx_table_.check_with_tx_data(ObTransID(ATOMIC_LOAD(&TEST_TX_ID)), op, 0)); + // 7,检查被测事务的tx data已经经过了deserialize以及从sstable读取过 + ASSERT_EQ(ATOMIC_LOAD(&LOAD_BIG_TX_DATA), true); + ASSERT_EQ(ATOMIC_LOAD(&READ_TEST_TX_FROM_SSTABLE), true); + } +} + +void TestBigTxData::minor_freeze_and_wait() +{ + int ret = OB_SUCCESS; + MTL_SWITCH(OB_SYS_TENANT_ID) { + int64_t affected_rows = 0; + int64_t retry_times = 40; + ObLSService *ls_service = MTL(ObLSService*); + ObLSHandle handle; + DO(ls_service->get_ls(ObLSID(1), handle, storage::ObLSGetMod::DEADLOCK_MOD)); + ObTxDataMemtableMgr *mgr = handle.get_ls()->tx_table_.tx_data_table_.memtable_mgr_; + ASSERT_NE(nullptr, mgr); + + int64_t head_before_freeze = -1; + int64_t tail_before_freeze = -1; + int64_t head_after_freeze = -1; + int64_t tail_after_freeze = -1; + while (--retry_times > 0) { + DO(mgr->get_memtable_range(head_before_freeze, tail_before_freeze)); + ASSERT_GE(head_before_freeze, 0); + ASSERT_GE(tail_before_freeze, 0); + if (head_before_freeze + 1 != tail_before_freeze) { + ::sleep(1); + cout << "waiting last minor freeze done ... " + << "head_before_freeze = " << head_before_freeze << " tail_before_freeze = " << tail_before_freeze << endl; + } else { + break; + } + } + ASSERT_EQ(head_before_freeze + 1, tail_before_freeze); + cout << "head_before_freeze : " << head_before_freeze << " tail_before_freeze" << tail_before_freeze << endl; + + // minor freeze once + DO(get_curr_simple_server().get_sql_proxy().write("alter system minor freeze", affected_rows)); + + retry_times = 60; + while (--retry_times > 0) + { + DO(handle.get_ls()->tx_table_.tx_data_table_.memtable_mgr_->get_memtable_range(head_after_freeze, tail_after_freeze)); + ASSERT_GE(head_after_freeze, 0); + ASSERT_GE(tail_after_freeze, 0); + if (head_after_freeze > head_before_freeze && tail_after_freeze > tail_before_freeze && head_after_freeze + 1 == tail_after_freeze) { + fprintf(stdout, + "head_after_freeze : %ld, head_before_freeze : %ld, tail_after_freeze : %ld, tail_before_freeze : %ld\n", + head_after_freeze, + head_before_freeze, + tail_after_freeze, + tail_before_freeze); + break; + } else { + ::sleep(1); + cout << "waiting this minor freeze done, head_after_freeze: " << head_after_freeze << " tail_after_freeze " << tail_after_freeze << endl; + } + } + ASSERT_GT(head_after_freeze, head_before_freeze); + ASSERT_GT(tail_after_freeze, tail_before_freeze); + ASSERT_EQ(head_after_freeze + 1, tail_after_freeze); + } +} +} // end unittest +} // end oceanbase + + +int main(int argc, char **argv) +{ + int c = 0; + int time_sec = 0; + char *log_level = (char*)"INFO"; + while(EOF != (c = getopt(argc,argv,"t:l:"))) { + switch(c) { + case 't': + time_sec = atoi(optarg); + break; + case 'l': + log_level = optarg; + oceanbase::unittest::ObSimpleClusterTestBase::enable_env_warn_log_ = false; + break; + default: + break; + } + } + oceanbase::unittest::init_log_and_gtest(argc, argv); + OB_LOGGER.set_log_level(log_level); + + oceanbase::unittest::RunCtx.time_sec_ = time_sec; + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/mittest/simple_server/test_mvcc_gc.cpp b/mittest/simple_server/test_mvcc_gc.cpp new file mode 100644 index 000000000..d1ed41598 --- /dev/null +++ b/mittest/simple_server/test_mvcc_gc.cpp @@ -0,0 +1,751 @@ +/** + * 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 protected public +#define private public + +#include "env/ob_simple_cluster_test_base.h" +#include "src/storage/ls/ob_ls_tablet_service.h" +#include "share/schema/ob_table_param.h" +#include "share/schema/ob_tenant_schema_service.h" +#include "storage/ob_dml_running_ctx.h" +#include "storage/access/ob_rows_info.h" +#include "storage/ob_relative_table.h" +#include "storage/compaction/ob_tenant_freeze_info_mgr.h" +#include "storage/concurrency_control/ob_multi_version_garbage_collector.h" + +static const char *TEST_FILE_NAME = "test_mvcc_gc"; + +namespace oceanbase +{ + +namespace concurrency_control +{ +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_RETRY_INTERVAL = 100_ms; +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_EXEC_INTERVAL = 1_s; +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_PRECISION = 1_ms; + +} // namespace concurrency_control + +namespace storage +{ +int ObLSTabletService::table_scan(ObTableScanIterator &iter, ObTableScanParam ¶m) +{ + int ret = OB_SUCCESS; + NG_TRACE(S_table_scan_begin); + ObTabletHandle data_tablet; + AllowToReadMgr::AllowToReadInfo read_info; + + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "not inited", K(ret), K_(is_inited)); + } else if (FALSE_IT(allow_to_read_mgr_.load_allow_to_read_info(read_info))) { + } else if (!read_info.allow_to_read()) { + ret = OB_REPLICA_NOT_READABLE; + STORAGE_LOG(WARN, "ls is not allow to read", K(ret), KPC(ls_)); + } else if (OB_FAIL(prepare_scan_table_param(param, *(MTL(ObTenantSchemaService*)->get_schema_service())))) { + STORAGE_LOG(WARN, "failed to prepare scan table param", K(ret), K(param)); + } else if (OB_FAIL(get_tablet_with_timeout(param.tablet_id_, data_tablet, param.timeout_))) { + STORAGE_LOG(WARN, "failed to check and get tablet", K(ret), K(param)); + } else if (OB_FAIL(inner_table_scan(data_tablet, iter, param))) { + STORAGE_LOG(WARN, "failed to do table scan", K(ret), KP(&iter), K(param)); + } else { + bool is_same = false; + allow_to_read_mgr_.check_read_info_same(read_info, is_same); + if (!is_same) { + ret = OB_REPLICA_NOT_READABLE; + STORAGE_LOG(WARN, "ls is not allow to read", K(ret), KPC(ls_), KP(&iter)); + } + } + NG_TRACE(S_table_scan_end); + + if (1002 == MTL_ID() && 200001 == param.tablet_id_.id()) { + DEBUG_SYNC(AFTER_TABLE_SCAN); + } + + return ret; +} + +int ObLSTabletService::insert_rows( + ObStoreCtx &ctx, + const ObDMLBaseParam &dml_param, + const common::ObIArray &column_ids, + common::ObNewRowIterator *row_iter, + int64_t &affected_rows) +{ + int ret = OB_SUCCESS; + NG_TRACE(S_insert_rows_begin); + ObTabletHandle tablet_handle; + int64_t afct_num = 0; + int64_t dup_num = 0; + ObTimeGuard timeguard(__func__, 3 * 1000 * 1000); + + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "not inited", K(ret), K_(is_inited)); + } else if (OB_UNLIKELY(!ctx.is_valid()) + || !ctx.is_write() + || OB_UNLIKELY(!dml_param.is_valid()) + || OB_UNLIKELY(column_ids.count() <= 0) + || OB_ISNULL(row_iter)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid args", K(ret), K(ctx), K(dml_param), K(column_ids), KP(row_iter)); + } else if (OB_FAIL(get_tablet_with_timeout( + ctx.tablet_id_, tablet_handle, dml_param.timeout_))) { + STORAGE_LOG(WARN, "failed to check and get tablet", K(ret), K(ctx.tablet_id_)); + } else { + ObArenaAllocator lob_allocator(ObModIds::OB_LOB_ACCESS_BUFFER, OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()); + ObDMLRunningCtx run_ctx(ctx, + dml_param, + ctx.mvcc_acc_ctx_.mem_ctx_->get_query_allocator(), + lob_allocator, + ObDmlFlag::DF_INSERT); + ObIAllocator &work_allocator = run_ctx.allocator_; + void *ptr = nullptr; + ObStoreRow *tbl_rows = nullptr; + int64_t row_count = 0; + //index of row that exists + int64_t row_count_first_bulk = 0; + bool first_bulk = true; + ObNewRow *rows = nullptr; + ObRowsInfo rows_info; + const ObRelativeTable &data_table = run_ctx.relative_table_; + + if (OB_FAIL(prepare_dml_running_ctx(&column_ids, nullptr, tablet_handle, run_ctx))) { + STORAGE_LOG(WARN, "failed to prepare dml running ctx", K(ret)); + } + + while (OB_SUCC(ret) && OB_SUCC(get_next_rows(row_iter, rows, row_count))) { + if (row_count <= 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "row_count should be greater than 0", K(ret)); + } else if (first_bulk) { + first_bulk = false; + row_count_first_bulk = row_count; + const ObTableReadInfo &full_read_info = tablet_handle.get_obj()->get_full_read_info(); + if (OB_FAIL(rows_info.init(data_table, ctx, full_read_info))) { + STORAGE_LOG(WARN, "Failed to init rows info", K(ret), K(data_table)); + } else if (OB_ISNULL(ptr = work_allocator.alloc(row_count * sizeof(ObStoreRow)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(ERROR, "fail to allocate memory", K(ret), K(row_count)); + } else { + tbl_rows = new (ptr) ObStoreRow[row_count]; + for (int64_t i = 0; i < row_count; i++) { + tbl_rows[i].flag_.set_flag(ObDmlFlag::DF_INSERT); + } + } + } + + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(tbl_rows)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected error, tbl_rows is NULL", K(ret), KP(tbl_rows)); + } else if (OB_FAIL(insert_rows_to_tablet(tablet_handle, run_ctx, rows, + row_count, rows_info, tbl_rows, afct_num, dup_num))) { + STORAGE_LOG(WARN, "insert to each tablets fail", K(ret)); + } + } + + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } + if (nullptr != ptr) { + work_allocator.free(ptr); + } + lob_allocator.reset(); + if (OB_SUCC(ret)) { + STORAGE_LOG(DEBUG, "succeeded to insert rows", K(ret)); + affected_rows = afct_num; + EVENT_ADD(STORAGE_INSERT_ROW_COUNT, afct_num); + } + } + NG_TRACE(S_insert_rows_end); + + if (1002 == MTL_ID() && 200001 == ctx.tablet_id_.id()) { + DEBUG_SYNC(AFTER_INSERT_ROWS); + } + + return ret; +} + +} // namespace storage + +namespace unittest +{ + +#define EXE_SQL(sql_str) \ + ASSERT_EQ(OB_SUCCESS, sql.assign(sql_str)); \ + ASSERT_EQ(OB_SUCCESS, sql_proxy.write(sql.ptr(), affected_rows)); + +#define EXE_SQL_FMT(...) \ + ASSERT_EQ(OB_SUCCESS, sql.assign_fmt(__VA_ARGS__)); \ + ASSERT_EQ(OB_SUCCESS, sql_proxy.write(sql.ptr(), affected_rows)); + +#define WRITE_SQL_BY_CONN(conn, sql_str) \ + ASSERT_EQ(OB_SUCCESS, sql.assign(sql_str)); \ + ASSERT_EQ(OB_SUCCESS, conn->execute_write(OB_SYS_TENANT_ID, sql.ptr(), affected_rows)); + +#define WRITE_SQL_FMT_BY_CONN(conn, ...) \ + ASSERT_EQ(OB_SUCCESS, sql.assign_fmt(__VA_ARGS__)); \ + ASSERT_EQ(OB_SUCCESS, conn->execute_write(OB_SYS_TENANT_ID, sql.ptr(), affected_rows)); + +#define READ_SQL_BY_CONN(conn, sql_str) \ + ASSERT_EQ(OB_SUCCESS, sql.assign(sql_str)); \ + ASSERT_EQ(OB_SUCCESS, conn->execute_read(OB_SYS_TENANT_ID, sql.ptr(), read_res)); + +class ObTestMvccGC : public ObSimpleClusterTestBase +{ +public: + ObTestMvccGC() : ObSimpleClusterTestBase(TEST_FILE_NAME) {} + void create_test_tenant(uint64_t &tenant_id) + { + TRANS_LOG(INFO, "create_tenant start"); + ASSERT_EQ(OB_SUCCESS, create_tenant()); + ASSERT_EQ(OB_SUCCESS, get_tenant_id(tenant_id)); + ASSERT_EQ(OB_SUCCESS, get_curr_simple_server().init_sql_proxy2()); + TRANS_LOG(INFO, "create_tenant end", K(tenant_id)); + } + void prepare_sys_env() + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL("alter system set debug_sync_timeout = '2000s'"); + } + void prepare_tenant_env() + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + int64_t affected_rows = 0; + ObSqlString sql; + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + WRITE_SQL_BY_CONN(connection, "set GLOBAL ob_trx_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set GLOBAL ob_trx_idle_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "set GLOBAL ob_query_timeout = 10000000000"); + WRITE_SQL_BY_CONN(connection, "alter system set undo_retention = 0"); + } + + void collect_garbage_collector_info(ObIArray &snapshots_info) + { + concurrency_control::ObMultiVersionGCSnapshotCollector collector(snapshots_info); + + ASSERT_EQ(OB_SUCCESS, MTL(concurrency_control::ObMultiVersionGarbageCollector *)->collect(collector)); + MVCC_LOG(INFO, "collect garbage collector info", K(snapshots_info)); + } + + void check_garbage_collector_info(concurrency_control::ObMultiVersionGCSnapshotFunctor& checker) + { + ASSERT_EQ(OB_SUCCESS, MTL(concurrency_control::ObMultiVersionGarbageCollector *)->collect(checker)); + MVCC_LOG(INFO, "check garbage collector info end"); + } + + void check_freeze_info_mgr(const int64_t expected_snapshot_version) + { + int64_t snapshot_version = 0; + share::SCN snapshot_version_for_txn; + ASSERT_EQ(OB_SUCCESS, MTL(storage::ObTenantFreezeInfoMgr *)-> + get_min_reserved_snapshot(ObTabletID(200001), 1, snapshot_version)); + snapshot_version_for_txn = MTL(concurrency_control::ObMultiVersionGarbageCollector *)-> + get_reserved_snapshot_for_active_txn(); + ASSERT_EQ(TRUE, expected_snapshot_version >= snapshot_version); + ASSERT_EQ(expected_snapshot_version, snapshot_version_for_txn.get_val_for_tx()); + } + + void wait_report() + { + // sleep 3 second + MVCC_LOG(INFO, "start to wait"); + usleep(3 * 1000 * 1000); + MVCC_LOG(INFO, "finish waiting"); + } + + void start_read_debug_sync(const char *name) + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL_FMT("set ob_global_debug_sync = 'AFTER_TABLE_SCAN wait_for %s'", name); + } + + void start_write_debug_sync(const char *name) + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL_FMT("set ob_global_debug_sync = 'AFTER_INSERT_ROWS wait_for %s'", name); + } + + void signal_debug_sync(const char *name) + { + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy(); + int64_t affected_rows = 0; + ObSqlString sql; + EXE_SQL_FMT("set ob_global_debug_sync = 'now signal %s'", name); + } + + void test_case1(); + void test_case2(); + void test_case3(); + void test_case4(); + void test_case5(); + void test_case6(); + void test_case7(); + + static bool is_disk_almost_full_; + static bool can_report_; + static bool is_refresh_fail_; +}; + +bool ObTestMvccGC::is_disk_almost_full_ = false; +bool ObTestMvccGC::can_report_ = true; +bool ObTestMvccGC::is_refresh_fail_ = false; + +void ObTestMvccGC::test_case1() +{ + // #CASE1: get snapshot with no active txn + wait_report(); + ObArray snapshots_info; + snapshots_info.reset(); + collect_garbage_collector_info(snapshots_info); + int64_t check_count = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_EQ(share::SCN::max_scn(), snapshot_version); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + return OB_SUCCESS; + }); + check_garbage_collector_info(functor); + ASSERT_EQ(4, check_count); +} + +void ObTestMvccGC::test_case2() +{ + // #CASE2: get snapshot with an ac = 0, RR txn + ObSqlString sql; + int64_t affected_rows = 0; + ObISQLClient::ReadResult read_res; + + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + WRITE_SQL_BY_CONN(connection, "SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ"); + WRITE_SQL_BY_CONN(connection, "begin;"); + READ_SQL_BY_CONN(connection, "select count(*) as cnt from test_mvcc_gc"); + + wait_report(); + int64_t check_count = 0; + int64_t active_snapshot = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count, + &active_snapshot](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_NE(share::SCN::max_scn(), snapshot_version); + active_snapshot = snapshot_version.get_val_for_tx(); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case2.1: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr)); + return OB_SUCCESS; + }); + check_garbage_collector_info(functor); + check_freeze_info_mgr(active_snapshot); + ASSERT_EQ(4, check_count); + + + WRITE_SQL_FMT_BY_CONN(connection, "insert into test_mvcc_gc values(1);"); + + wait_report(); + check_count = 0; + active_snapshot = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor2( + [&check_count, + &active_snapshot](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_NE(share::SCN::max_scn(), snapshot_version); + active_snapshot = snapshot_version.get_val_for_tx(); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case2.2: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr)); + return OB_SUCCESS; + }); + check_garbage_collector_info(functor2); + check_freeze_info_mgr(active_snapshot); + ASSERT_EQ(4, check_count); + WRITE_SQL_BY_CONN(connection, "commit;"); +} + +void ObTestMvccGC::test_case3() +{ + // #CASE3: get snapshot with an ac = 0, RC txn + ObSqlString sql; + int64_t affected_rows = 0; + ObISQLClient::ReadResult read_res; + + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + WRITE_SQL_BY_CONN(connection, "SET SESSION TRANSACTION ISOLATION LEVEL READ COMMITTED"); + WRITE_SQL_BY_CONN(connection, "begin;"); + + READ_SQL_BY_CONN(connection, "select count(*) as cnt from test_mvcc_gc"); + + wait_report(); + int64_t check_count = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_EQ(share::SCN::max_scn(), snapshot_version); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case3.1: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr)); + return OB_SUCCESS; + }); + + check_garbage_collector_info(functor); + ASSERT_EQ(4, check_count); + + WRITE_SQL_FMT_BY_CONN(connection, "insert into test_mvcc_gc values(1);"); + + wait_report(); + check_count = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor2( + [&check_count](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_EQ(share::SCN::max_scn(), snapshot_version); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case3.2: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr)); + return OB_SUCCESS; + }); + check_garbage_collector_info(functor2); + ASSERT_EQ(4, check_count); + WRITE_SQL_BY_CONN(connection, "commit;"); +} + +void ObTestMvccGC::test_case4() +{ + // #CASE4: get snapshot with an ac = 1 txn + ObSqlString sql; + int64_t affected_rows = 0; + ObISQLClient::ReadResult read_res; + + start_read_debug_sync("test_case4_1"); + + std::thread *thread = new std::thread( + [this]() -> void { + ObSqlString sql; + int64_t affected_rows = 0; + ObISQLClient::ReadResult read_res; + + common::ObMySQLProxy &sql_proxy = this->get_curr_simple_server().get_sql_proxy2(); + sqlclient::ObISQLConnection *connection = nullptr; + ASSERT_EQ(OB_SUCCESS, sql_proxy.acquire(connection)); + ASSERT_NE(nullptr, connection); + + TRANS_LOG(INFO, "start read"); + READ_SQL_BY_CONN(connection, "select count(*) as cnt from test_mvcc_gc"); + TRANS_LOG(INFO, "end read"); + + TRANS_LOG(INFO, "start write"); + WRITE_SQL_BY_CONN(connection, "insert into test_mvcc_gc values(1)"); + TRANS_LOG(INFO, "end write"); + }); + // wait thread execution + usleep(5 * 1000 * 1000); + + wait_report(); + int64_t check_count = 0; + int64_t active_snapshot = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count, + &active_snapshot](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_NE(share::SCN::max_scn(), snapshot_version); + active_snapshot = snapshot_version.get_val_for_tx(); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case4.1: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr)); + return OB_SUCCESS; + }); + check_garbage_collector_info(functor); + check_freeze_info_mgr(active_snapshot); + ASSERT_EQ(4, check_count); + + start_write_debug_sync("test_case4_2"); + signal_debug_sync("test_case4_1"); + + // wait thread execution + usleep(5 * 1000 * 1000); + + wait_report(); + check_count = 0; + active_snapshot = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor2( + [&check_count, + &active_snapshot](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_NE(share::SCN::max_scn(), snapshot_version); + active_snapshot = snapshot_version.get_val_for_tx(); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case4.2: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr)); + return OB_SUCCESS; + }); + check_garbage_collector_info(functor2); + check_freeze_info_mgr(active_snapshot); + ASSERT_EQ(4, check_count); + + signal_debug_sync("test_case4_2"); + thread->join(); +} + +void ObTestMvccGC::test_case5() +{ + // #CASE5: test disk is full + is_disk_almost_full_ = true; + + wait_report(); + wait_report(); + int64_t check_count = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::DISABLED_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_EQ(share::SCN::max_scn(), snapshot_version); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case5: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr), K(status)); + return OB_SUCCESS; + }); + + check_garbage_collector_info(functor); + check_freeze_info_mgr(INT64_MAX); + ASSERT_EQ(4, check_count); + + is_disk_almost_full_ = false; + wait_report(); +} + +void ObTestMvccGC::test_case6() +{ + // #CASE5: test donot report + can_report_ = false; + wait_report(); + wait_report(); + int64_t check_count = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + TRANS_LOG(INFO, "test_case6: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr), K(status)); + return OB_SUCCESS; + }); + + check_garbage_collector_info(functor); + ASSERT_EQ(0, check_count); + + can_report_ = true; + wait_report(); +} + +void ObTestMvccGC::test_case7() +{ + // #CASE5: test donot refresh + is_refresh_fail_ = true; + wait_report(); + int64_t check_count = 0; + concurrency_control::ObMultiVersionGCSnapshotOperator functor( + [&check_count](const share::SCN snapshot_version, + const concurrency_control::ObMultiVersionSnapshotType snapshot_type, + const concurrency_control::ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + check_count++; + EXPECT_EQ(concurrency_control::ObMultiVersionGCStatus::NORMAL_GC_STATUS, status); + if (concurrency_control::ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT == snapshot_type) { + EXPECT_EQ(share::SCN::max_scn(), snapshot_version); + } else { + EXPECT_EQ(true, abs((snapshot_version.get_val_for_tx() / 1000) - create_time) < 1 * 1000 * 1000); + } + TRANS_LOG(INFO, "test_case7: check gc snapshot info", K(snapshot_version), + K(snapshot_type), K(create_time), K(addr), K(status)); + return OB_SUCCESS; + }); + + check_garbage_collector_info(functor); + check_freeze_info_mgr(INT64_MAX); + ASSERT_EQ(4, check_count); + + is_refresh_fail_ = false; + wait_report(); +} + +TEST_F(ObTestMvccGC, test_basic_mvcc_gc) +{ + ObSqlString sql; + int64_t affected_rows = 0; + + // ============================== Phase1. create tenant and table ============================== + TRANS_LOG(INFO, "create tenant start"); + uint64_t tenant_id = 0; + create_test_tenant(tenant_id); + TRANS_LOG(INFO, "create tenant end"); + + prepare_sys_env(); + + share::ObTenantSwitchGuard tenant_guard; + ASSERT_EQ(OB_SUCCESS, tenant_guard.switch_to(tenant_id)); + + TRANS_LOG(INFO, "create table start"); + common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); + EXE_SQL("create table test_mvcc_gc (a int)"); + // wait minor freeze when create table + usleep(10 * 1000 * 1000); + TRANS_LOG(INFO, "create_table end"); + + prepare_tenant_env(); + + // #CASE1: get snapshot with no active txn + test_case1(); + + // #CASE5: check when refresh fail too long + test_case7(); + + // #CASE2: get snapshot with an ac = 0, RR txn + test_case2(); + + // #CASE5: check when report fail too long + test_case6(); + + // #CASE3: get snapshot with an ac = 0, RC txn + test_case3(); + + // #CASE5: check when disk is full + test_case5(); + + // #CASE4: get snapshot with an ac = 1 txn + test_case4(); +} + +} // namespace unittest + +namespace concurrency_control +{ + +bool ObMultiVersionGarbageCollector::can_report() +{ + return unittest::ObTestMvccGC::can_report_; +} + +bool ObMultiVersionGarbageCollector::is_refresh_fail() +{ + return unittest::ObTestMvccGC::is_refresh_fail_; +} + +int ObMultiVersionGarbageCollector::is_disk_almost_full_(bool &is_almost_full) +{ + int ret = OB_SUCCESS; + + if (unittest::ObTestMvccGC::is_disk_almost_full_) { + is_almost_full = true; + } else { + is_almost_full = false; + } + + return ret; +} + +} // namespace concurrency_control + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + oceanbase::unittest::init_log_and_gtest(argc, argv); + OB_LOGGER.set_log_level("info"); + ::testing::InitGoogleTest(&argc, argv); + + return RUN_ALL_TESTS(); +} diff --git a/mittest/simple_server/test_replay_from_middle.cpp b/mittest/simple_server/test_replay_from_middle.cpp index f50233bb1..f2f73ed8e 100644 --- a/mittest/simple_server/test_replay_from_middle.cpp +++ b/mittest/simple_server/test_replay_from_middle.cpp @@ -460,6 +460,7 @@ void ObReplayRestartTest::restart_test() // 场景一: keep alive日志没有被回放,min_start_scn为初始值状态,跳过计算upper_trans_version ASSERT_EQ(SCN::min_scn(), tx_data_table->calc_upper_info_.min_start_scn_in_ctx_); upper_trans_version.set_min(); + FLOG_INFO("get upper trans version, situation 1:", K(SSTABLE_END_SCN)); ASSERT_EQ(OB_SUCCESS, tx_data_table->get_upper_trans_version_before_given_scn(SSTABLE_END_SCN, upper_trans_version)); } @@ -478,6 +479,7 @@ void ObReplayRestartTest::restart_test() } } upper_trans_version.set_min(); + FLOG_INFO("get upper trans version, situation 2:", K(SSTABLE_END_SCN)); ASSERT_EQ(OB_SUCCESS, tx_data_table->get_upper_trans_version_before_given_scn(SSTABLE_END_SCN, upper_trans_version)); ASSERT_GE(upper_trans_version, SCN::max_scn()); @@ -526,8 +528,12 @@ void ObReplayRestartTest::restart_test() ASSERT_GT(max_decided_scn, tx_data_table->calc_upper_info_.keep_alive_scn_); upper_trans_version.set_min(); + FLOG_INFO("get upper trans version, situation 3:", K(SSTABLE_END_SCN)); ASSERT_EQ(OB_SUCCESS, tx_data_table->get_upper_trans_version_before_given_scn(SSTABLE_END_SCN, upper_trans_version)); + + + ::sleep(10); STORAGE_LOG(INFO, "finish restart test", K(upper_trans_version), K(SSTABLE_END_SCN), K(tx_data_table->calc_upper_info_)); ASSERT_LT(upper_trans_version, SCN::max_scn()); } diff --git a/mittest/simple_server/test_tx_data_table_mit.cpp b/mittest/simple_server/test_tx_data_table_mit.cpp index 59a4e9e9c..9249246f7 100644 --- a/mittest/simple_server/test_tx_data_table_mit.cpp +++ b/mittest/simple_server/test_tx_data_table_mit.cpp @@ -50,6 +50,7 @@ namespace oceanbase using namespace transaction; using namespace storage; using namespace palf; +using namespace share; namespace unittest { @@ -174,21 +175,19 @@ void ObTxDataTableTest::check_start_tx_scn(ObTxDataTable *tx_data_table) // MAKE_TENANT_SWITCH_SCOPE_GUARD(guard); // guard.switch_to(RunCtx.tenant_id_); - // share::SCN pre_start_tx_scn = share::SCN::min_scn(); - // while (!ATOMIC_LOAD(&stop)) { - // share::SCN start_tx_scn = share::SCN::min_scn(); - // share::SCN recycle_scn = share::SCN::min_scn(); - // ASSERT_EQ(OB_SUCCESS, tx_data_table->get_start_tx_scn(start_tx_scn)); - // STORAGE_LOG(INFO, "GENGLI ", K(tx_data_table->get_ls_id()), K(pre_start_tx_scn), K(start_tx_scn)); - // ASSERT_LE(pre_start_tx_scn, start_tx_scn); - // ASSERT_EQ(OB_SUCCESS, tx_data_table->get_recycle_scn(recycle_scn)); - // STORAGE_LOG(INFO, "GENGLI ", K(tx_data_table->get_ls_id()), K(recycle_scn), K(start_tx_scn)); - // ASSERT_LE(start_tx_scn, recycle_scn); - // pre_start_tx_scn = start_tx_scn; - // OB_LOG(INFO, "check start tx scn : ", K(start_tx_scn), K(recycle_scn)); + SCN pre_start_tx_scn = SCN::min_scn(); + while (!ATOMIC_LOAD(&stop)) { + SCN start_tx_scn = SCN::min_scn(); + SCN recycle_scn = SCN::min_scn(); + ASSERT_EQ(OB_SUCCESS, tx_data_table->get_start_tx_scn(start_tx_scn)); + ASSERT_LE(pre_start_tx_scn, start_tx_scn); + ASSERT_EQ(OB_SUCCESS, tx_data_table->get_recycle_scn(recycle_scn)); + ASSERT_LE(start_tx_scn, recycle_scn); + pre_start_tx_scn = start_tx_scn; + OB_LOG(INFO, "check start tx scn : ", K(start_tx_scn), K(recycle_scn)); // ::sleep(1); - // } + } } void ObTxDataTableTest::check_tx_data_minor_succeed() @@ -316,6 +315,7 @@ public: void ObTxDataTableRestartTest::select_existed_data() { + sleep(10); common::ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2(); int ret = OB_SUCCESS; diff --git a/src/objit/include/objit/common/ob_item_type.h b/src/objit/include/objit/common/ob_item_type.h index da5b9ee09..698f6fe7a 100755 --- a/src/objit/include/objit/common/ob_item_type.h +++ b/src/objit/include/objit/common/ob_item_type.h @@ -960,6 +960,7 @@ typedef enum ObItemType T_INDEX_TABLE_ID, T_TABLET_ID, T_TENANT_TABLET, + T_TENANT_LS_TABLET, T_VIRTUAL_COLUMN_ID, T_DATABASE_ID, T_REPLICA_NUM, diff --git a/src/observer/ob_service.cpp b/src/observer/ob_service.cpp index 3103e0a31..4c40be2e8 100644 --- a/src/observer/ob_service.cpp +++ b/src/observer/ob_service.cpp @@ -945,58 +945,12 @@ int ObService::minor_freeze(const obrpc::ObMinorFreezeArg &arg, } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); + } else if (arg.ls_id_.is_valid() || arg.tablet_id_.is_valid()) { + ret = handle_ls_freeze_req_(arg); + } else if (arg.tenant_ids_.count() > 0) { + ret = handle_tenant_freeze_req_(arg); } else { - if (arg.tablet_id_.is_valid()) { - // minor feeeze tablet - if (1 == arg.tenant_ids_.count()) { - uint64_t tenant_id = arg.tenant_ids_.at(0); - if (OB_UNLIKELY(OB_FAIL(tablet_freeze(tenant_id, arg.tablet_id_)))) { - LOG_WARN("fail to freeze tablet", K(ret), K(tenant_id), K(arg.tablet_id_)); - } - } else { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("only one tenant is needed", K(ret), K(arg.tenant_ids_), K(arg.tablet_id_)); - } - } else if (arg.tenant_ids_.count() > 0) { - // minor freeze tenants - for (int i = 0; i < arg.tenant_ids_.count(); ++i) { - int tmp_ret = OB_SUCCESS; - uint64_t tenant_id = arg.tenant_ids_.at(i); - if (OB_UNLIKELY(OB_SUCCESS != (tmp_ret = tenant_freeze(tenant_id)))) { - LOG_WARN("fail to freeze tenant", K(tmp_ret), K(tenant_id)); - } - // record the first error code - if (OB_SUCCESS != tmp_ret && OB_SUCC(ret)) { - ret = tmp_ret; - } - } - } else { - // for minor freeze server - // freeze all tenants - if (OB_ISNULL(GCTX.omt_)) { - ret = OB_ERR_UNEXPECTED; - SERVER_LOG(WARN, "failed to get multi tenant from GCTX", K(ret)); - } else { - omt::TenantIdList all_tenants; - GCTX.omt_->get_tenant_ids(all_tenants); - for (int i = 0; i < all_tenants.size(); ++i) { - int tmp_ret = OB_SUCCESS; - uint64_t tenant_id = all_tenants[i]; - if (OB_UNLIKELY(OB_SUCCESS != (tmp_ret = tenant_freeze(tenant_id)))) { - if (OB_TENANT_NOT_IN_SERVER == tmp_ret) { - LOG_INFO("skip freeze stopped tenant", K(tmp_ret), K(tenant_id)); - tmp_ret = OB_SUCCESS; - } else { - LOG_WARN("fail to freeze tenant", K(tmp_ret), K(tenant_id)); - } - } - // record the first error code - if (OB_SUCCESS != tmp_ret && OB_SUCC(ret)) { - ret = tmp_ret; - } - } - } - } + ret = handle_server_freeze_req_(arg); } result = ret; @@ -1005,7 +959,65 @@ int ObService::minor_freeze(const obrpc::ObMinorFreezeArg &arg, return ret; } -int ObService::tablet_freeze(const uint64_t tenant_id, const common::ObTabletID &tablet_id) +int ObService::handle_server_freeze_req_(const obrpc::ObMinorFreezeArg &arg) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(GCTX.omt_)) { + ret = OB_ERR_UNEXPECTED; + SERVER_LOG(WARN, "failed to get multi tenant from GCTX", K(ret)); + } else { + omt::TenantIdList all_tenants; + GCTX.omt_->get_tenant_ids(all_tenants); + for (int i = 0; i < all_tenants.size(); ++i) { + int tmp_ret = OB_SUCCESS; + uint64_t tenant_id = all_tenants[i]; + if (OB_UNLIKELY(OB_SUCCESS != (tmp_ret = tenant_freeze_(tenant_id)))) { + if (OB_TENANT_NOT_IN_SERVER == tmp_ret) { + LOG_INFO("skip freeze stopped tenant", K(tmp_ret), K(tenant_id)); + tmp_ret = OB_SUCCESS; + } else { + LOG_WARN("fail to freeze tenant", K(tmp_ret), K(tenant_id)); + } + } + // record the first error code + if (OB_SUCCESS != tmp_ret && OB_SUCC(ret)) { + ret = tmp_ret; + } + } + } + return ret; +} + +int ObService::handle_tenant_freeze_req_(const obrpc::ObMinorFreezeArg &arg) +{ + int ret = OB_SUCCESS; + for (int i = 0; i < arg.tenant_ids_.count(); ++i) { + int tmp_ret = OB_SUCCESS; + uint64_t tenant_id = arg.tenant_ids_.at(i); + if (OB_UNLIKELY(OB_SUCCESS != (tmp_ret = tenant_freeze_(tenant_id)))) { + LOG_WARN("fail to freeze tenant", K(tmp_ret), K(tenant_id)); + } + // record the first error code + if (OB_SUCCESS != tmp_ret && OB_SUCC(ret)) { + ret = tmp_ret; + } + } + return ret; +} + +int ObService::handle_ls_freeze_req_(const obrpc::ObMinorFreezeArg &arg) +{ + int ret = OB_SUCCESS; + if (1 != arg.tenant_ids_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("only one tenant is needed", K(ret), K(arg.tenant_ids_), K(arg.tablet_id_)); + } else if (OB_FAIL(ls_freeze_(arg.tenant_ids_.at(0), arg.ls_id_, arg.tablet_id_))) { + LOG_WARN("fail to freeze tablet", K(ret), K(arg)); + } + return ret; +} + +int ObService::ls_freeze_(const uint64_t tenant_id, const share::ObLSID &ls_id, const common::ObTabletID &tablet_id) { int ret = OB_SUCCESS; @@ -1017,10 +1029,24 @@ int ObService::tablet_freeze(const uint64_t tenant_id, const common::ObTabletID if (OB_ISNULL(freezer = MTL(storage::ObTenantFreezer*))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ObTenantFreezer shouldn't be null", K(ret), K(tenant_id)); - } else if (OB_FAIL(freezer->tablet_freeze(tablet_id))) { - LOG_WARN("fail to freeze tablet", K(ret), K(tenant_id), K(tablet_id)); + } else if (tablet_id.is_valid()) { + // tablet freeze + if (OB_FAIL(freezer->tablet_freeze(ls_id, tablet_id))) { + LOG_WARN("fail to freeze tablet", K(ret), K(tenant_id), K(ls_id), K(tablet_id)); + } else { + LOG_INFO("succeed to freeze tablet", K(ret), K(tenant_id), K(ls_id), K(tablet_id)); + } } else { - LOG_INFO("succeed to freeze tablet", K(ret), K(tenant_id), K(tablet_id)); + // logstream freeze + if (OB_FAIL(freezer->ls_freeze(ls_id))) { + if (OB_ENTRY_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to freeze ls", K(ret), K(tenant_id), K(ls_id), K(tablet_id)); + } + } else { + LOG_INFO("succeed to freeze ls", K(ret), K(tenant_id), K(ls_id), K(tablet_id)); + } } } } @@ -1028,7 +1054,7 @@ int ObService::tablet_freeze(const uint64_t tenant_id, const common::ObTabletID return ret; } -int ObService::tenant_freeze(const uint64_t tenant_id) +int ObService::tenant_freeze_(const uint64_t tenant_id) { int ret = OB_SUCCESS; diff --git a/src/observer/ob_service.h b/src/observer/ob_service.h index 738c018c5..14131208a 100644 --- a/src/observer/ob_service.h +++ b/src/observer/ob_service.h @@ -238,8 +238,12 @@ private: int register_self(); int check_server_empty(const obrpc::ObCheckServerEmptyArg &arg, const bool wait_log_scan, bool &server_empty); - int tenant_freeze(const uint64_t tenant_id); - int tablet_freeze(const uint64_t tenant_id, const common::ObTabletID &tablet_id); + + int handle_server_freeze_req_(const obrpc::ObMinorFreezeArg &arg); + int handle_tenant_freeze_req_(const obrpc::ObMinorFreezeArg &arg); + int handle_ls_freeze_req_(const obrpc::ObMinorFreezeArg &arg); + int tenant_freeze_(const uint64_t tenant_id); + int ls_freeze_(const uint64_t tenant_id, const share::ObLSID &ls_id, const common::ObTabletID &tablet_id); private: bool inited_; bool in_register_process_; diff --git a/src/observer/omt/ob_multi_tenant.cpp b/src/observer/omt/ob_multi_tenant.cpp index 40aee4e54..e78842d5e 100644 --- a/src/observer/omt/ob_multi_tenant.cpp +++ b/src/observer/omt/ob_multi_tenant.cpp @@ -48,6 +48,7 @@ #include "storage/tx_storage/ob_ls_service.h" #include "storage/tx_storage/ob_access_service.h" #include "storage/tx_storage/ob_tenant_freezer.h" +#include "storage/concurrency_control/ob_multi_version_garbage_collector.h" #include "storage/tx/ob_xa_service.h" #include "storage/tx/ob_tx_loop_worker.h" #include "storage/tx/ob_timestamp_service.h" @@ -100,6 +101,7 @@ using namespace oceanbase::storage::checkpoint; using namespace oceanbase::obmysql; using namespace oceanbase::sql; using namespace oceanbase::sql::dtl; +using namespace oceanbase::concurrency_control; using namespace oceanbase::transaction; using namespace oceanbase::transaction::tablelock; using namespace oceanbase::logservice; @@ -317,6 +319,7 @@ int ObMultiTenant::init(ObAddr myaddr, MTL_BIND2(mtl_new_default, ObDASIDService::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObAccessService::mtl_init, nullptr, mtl_stop_default, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObCheckPointService::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); + MTL_BIND2(mtl_new_default, ObMultiVersionGarbageCollector::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObUDRMgr::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND(ObPxPools::mtl_init, ObPxPools::mtl_destroy); diff --git a/src/observer/table/ob_table_service.cpp b/src/observer/table/ob_table_service.cpp index 032d33c8e..30199d44e 100644 --- a/src/observer/table/ob_table_service.cpp +++ b/src/observer/table/ob_table_service.cpp @@ -303,6 +303,7 @@ int ObTableService::do_put(ObTableServiceCtx &ctx, const ObTableOperation &table dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); if (OB_FAIL(access_service->put_rows( ls_id, tablet_id, @@ -446,6 +447,7 @@ int ObTableService::multi_put(ObTableServiceCtx &ctx, dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); const int64_t N = batch_operation.count(); NG_TRACE_EXT(insertup_calc_new_row, OB_ID(input_count), N); for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) { @@ -824,6 +826,7 @@ int ObTableService::execute_delete(ObTableServiceGetCtx &ctx, const ObTableOpera dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); if (OB_FAIL(access_service->delete_rows( ls_id, tablet_id, @@ -920,6 +923,7 @@ int ObTableService::multi_delete(ObTableServiceGetCtx &ctx, const ObTableBatchOp dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); int64_t result_idx = 0; while(OB_SUCC(ret) && !delete_row_iter.has_finished()) { affected_rows = 0; @@ -1012,6 +1016,7 @@ int ObTableService::execute_insert( dml_param.schema_version_ = insert_iter.get_schema_version(); dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); if (ObTableOperationType::INSERT_OR_UPDATE == table_operation.type()) { // use insert row with duplicate rows ObSEArray rowkey_column_ids; @@ -1122,6 +1127,7 @@ int ObTableService::multi_insert(ObTableServiceCtx &ctx, const ObTableBatchOpera dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); const int64_t N = batch_operation.count(); for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { insert_iter.continue_iter(); @@ -1324,6 +1330,7 @@ int ObTableService::execute_replace(ObTableServiceCtx &ctx, const ObTableOperati dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); ObIArray &full_column_ids = replace_iter.get_column_ids(); const int64_t rowkey_column_cnt = table_operation.entity().get_rowkey_size(); ObSEArray rowkey_column_ids; @@ -1375,6 +1382,7 @@ int ObTableService::multi_replace(ObTableServiceCtx &ctx, const ObTableBatchOper dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); // fill column ids ObIArray &column_ids = replace_iter.get_column_ids(); ObSEArray rowkey_column_ids; @@ -1451,6 +1459,7 @@ int ObTableService::execute_update(ObTableServiceGetCtx &ctx, dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); if (OB_FAIL(access_service->update_rows( ls_id, tablet_id, @@ -1550,6 +1559,7 @@ int ObTableService::execute_increment_by_update( dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); if (OB_FAIL(access_service->update_rows( ls_id, tablet_id, @@ -1659,6 +1669,7 @@ int ObTableService::multi_update(ObTableServiceGetCtx &ctx, const ObTableBatchOp dml_param.encrypt_meta_ = &dml_param.encrypt_meta_legacy_; dml_param.table_param_ = &table_param; dml_param.snapshot_ = ctx.param_.processor_->get_tx_snapshot(); + dml_param.write_flag_.set_is_table_api(); while(OB_SUCC(ret) && !update_row_iter.has_finished()) { affected_rows = 0; update_row_iter.continue_iter(); diff --git a/src/rootserver/ob_root_minor_freeze.cpp b/src/rootserver/ob_root_minor_freeze.cpp index 71309939d..bbc25edf7 100644 --- a/src/rootserver/ob_root_minor_freeze.cpp +++ b/src/rootserver/ob_root_minor_freeze.cpp @@ -154,10 +154,7 @@ int ObRootMinorFreeze::get_tenant_server_list(uint64_t tenant_id, return ret; } -int ObRootMinorFreeze::try_minor_freeze(const ObIArray &tenant_ids, - const ObIArray &server_list, - const common::ObZone &zone, - const common::ObTabletID &tablet_id) const +int ObRootMinorFreeze::try_minor_freeze(const obrpc::ObRootMinorFreezeArg &arg) const { int ret = OB_SUCCESS; if (!inited_) { @@ -165,28 +162,26 @@ int ObRootMinorFreeze::try_minor_freeze(const ObIArray &tenant_ids, LOG_WARN("ObRootMinorFreeze not init", K(ret)); } else { ParamsContainer params; - if (tablet_id.is_valid()) { - if (1 == tenant_ids.count()) { - if (OB_FAIL(init_params_by_tablet_id(tenant_ids.at(0), - tablet_id, - params))) { + if ((arg.ls_id_.is_valid() && arg.ls_id_.id() > 0) || arg.tablet_id_.is_valid()) { + if (1 == arg.tenant_ids_.count()) { + if (OB_FAIL(init_params_by_ls_or_tablet(arg.tenant_ids_.at(0), arg.ls_id_, arg.tablet_id_, params))) { LOG_WARN("fail to init param by tablet_id"); } } else { ret = OB_NOT_SUPPORTED; - LOG_WARN("only one tenant is required for tablet_freeze", K(ret), K(tablet_id), K(tenant_ids)); + LOG_WARN("only one tenant is required for tablet_freeze", K(ret), K(arg)); } - } else if (tenant_ids.count() > 0) { - if (OB_FAIL(init_params_by_tenant(tenant_ids, zone, server_list, params))) { - LOG_WARN("fail to init param by tenant, ", K(tenant_ids), K(tablet_id), K(ret)); + } else if (arg.tenant_ids_.count() > 0) { + if (OB_FAIL(init_params_by_tenant(arg.tenant_ids_, arg.zone_, arg.server_list_, params))) { + LOG_WARN("fail to init param by tenant, ", K(ret), K(arg)); } - } else if (server_list.count() == 0 && zone.size() > 0) { - if (OB_FAIL(init_params_by_zone(zone, params))) { - LOG_WARN("fail to init param by tenant, ", K(tenant_ids), K(ret)); + } else if (arg.server_list_.count() == 0 && arg.zone_.size() > 0) { + if (OB_FAIL(init_params_by_zone(arg.zone_, params))) { + LOG_WARN("fail to init param by zone, ", K(ret), K(arg)); } } else { - if (OB_FAIL(init_params_by_server(server_list, params))) { - LOG_WARN("fail to init param by server, ", K(server_list), K(ret)); + if (OB_FAIL(init_params_by_server(arg.server_list_, params))) { + LOG_WARN("fail to init param by server, ", K(ret), K(arg)); } } @@ -234,6 +229,7 @@ int ObRootMinorFreeze::do_minor_freeze(const ParamsContainer ¶ms) const if (0 != failure_cnt && OB_CANCELED != ret) { ret = OB_PARTIAL_FAILED; + LOG_WARN("minor freeze partial failed", KR(ret), K(failure_cnt)); } return ret; @@ -262,48 +258,53 @@ int ObRootMinorFreeze::is_server_belongs_to_zone(const ObAddr &addr, return ret; } -int ObRootMinorFreeze::init_params_by_tablet_id(const uint64_t tenant_id, - const common::ObTabletID &tablet_id, - ParamsContainer ¶ms) const +int ObRootMinorFreeze::init_params_by_ls_or_tablet(const uint64_t tenant_id, + share::ObLSID ls_id, + const common::ObTabletID &tablet_id, + ParamsContainer ¶ms) const { int ret = OB_SUCCESS; - const int64_t cluster_id = GCONF.cluster_id; - share::ObLSID ls_id; - int64_t expire_renew_time = INT64_MAX; - bool is_cache_hit = false; + const int64_t expire_renew_time = INT64_MAX; share::ObLSLocation location; - if (OB_UNLIKELY(nullptr == GCTX.location_service_)) { + bool is_cache_hit = false; + if (OB_UNLIKELY(OB_ISNULL(GCTX.location_service_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("location service ptr is null", KR(ret)); - } else if (OB_FAIL(GCTX.location_service_->get(tenant_id, - tablet_id, - INT64_MAX, - is_cache_hit, - ls_id))) { - LOG_WARN("fail to get ls id according to tablet_id", K(ret), K(tenant_id), K(tablet_id)); - } else if (OB_FAIL(GCTX.location_service_->get(cluster_id, - tenant_id, - ls_id, - expire_renew_time, - is_cache_hit, - location))) { - LOG_WARN("fail to get ls location", KR(ret), K(cluster_id), K(tenant_id), K(ls_id), K(tablet_id)); + } else if (tablet_id.is_valid() && !ls_id.is_valid()) { + // get ls id by tablet_id + if (tablet_id.is_ls_inner_tablet()) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("can not minor freeze inner tablet without specifying ls id", K(tenant_id), K(ls_id), K(tablet_id)); + } else if (OB_FAIL(GCTX.location_service_->get(tenant_id, tablet_id, expire_renew_time, is_cache_hit, ls_id))) { + LOG_WARN("fail to get ls id according to tablet_id", K(ret), K(tenant_id), K(tablet_id)); + } + } + + if (OB_FAIL(ret)) { + } else if (ls_id.is_valid()) { + // get ls location by ls_id + if (OB_FAIL(GCTX.location_service_->get( + GCONF.cluster_id, tenant_id, ls_id, expire_renew_time, is_cache_hit, location))) { + LOG_WARN("get ls location failed", KR(ret), K(tenant_id), K(ls_id), K(tablet_id)); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid ls_id or tablet_id", KR(ret), K(ls_id), K(tablet_id)); + } + + if (OB_FAIL(ret)) { } else { const ObIArray &ls_locations = location.get_replica_locations(); for (int i = 0; i < ls_locations.count() && OB_SUCC(ret); ++i) { const ObAddr &server = ls_locations.at(i).get_server(); if (is_server_alive(server)) { - if (OB_FAIL(params.add_tenant_server(tenant_id, - tablet_id, - server))) { - LOG_WARN("fail to add tenant & server, ", K(ret), K(tenant_id), K(ls_id), - K(tablet_id)); + if (OB_FAIL(params.push_back_param(server, tenant_id, ls_id, tablet_id))) { + LOG_WARN("fail to add tenant & server, ", K(ret), K(tenant_id), K(ls_id), K(tablet_id)); } } else { int tmp_ret = OB_SERVER_NOT_ACTIVE; - LOG_WARN("server not alive or invalid", "server", server, K(tmp_ret), K(tenant_id), - K(ls_id), K(tablet_id)); + LOG_WARN("server not alive or invalid", "server", server, K(tmp_ret), K(tenant_id), K(ls_id), K(tablet_id)); } } } @@ -318,16 +319,12 @@ int ObRootMinorFreeze::init_params_by_tenant(const ObIArray &tenant_id { int ret = OB_SUCCESS; ObSEArray target_server_list; - common::ObTabletID tablet_id; - tablet_id.reset(); for (int i = 0; i < tenant_ids.count() && OB_SUCC(ret); ++i) { if (server_list.count() > 0) { for (int j = 0; j < server_list.count() && OB_SUCC(ret); ++j) { if (is_server_alive(server_list.at(j))) { - if (OB_FAIL(params.add_tenant_server(tenant_ids.at(i), - tablet_id, - server_list.at(j)))) { + if (OB_FAIL(params.push_back_param(server_list.at(j), tenant_ids.at(i)))) { LOG_WARN("fail to add tenant & server, ", K(ret)); } } else { @@ -345,9 +342,7 @@ int ObRootMinorFreeze::init_params_by_tenant(const ObIArray &tenant_id const ObAddr &server = target_server_list.at(j); if (OB_FAIL(is_server_belongs_to_zone(server, zone, server_in_zone))) { LOG_WARN("fail to check server", K(ret)); - } else if (server_in_zone && OB_FAIL(params.add_tenant_server(tenant_ids.at(i), - tablet_id, - server))) { + } else if (server_in_zone && OB_FAIL(params.push_back_param(server, tenant_ids.at(i)))) { LOG_WARN("fail to add tenant & server", K(ret)); } } @@ -374,7 +369,7 @@ int ObRootMinorFreeze::init_params_by_zone(const ObZone &zone, LOG_WARN("empty zone or invalid", K(zone), K(ret)); } else { for (int i = 0; i < target_server_list.count() && OB_SUCC(ret); ++i) { - if (OB_FAIL(params.add_server(target_server_list.at(i)))) { + if (OB_FAIL(params.push_back_param(target_server_list.at(i)))) { LOG_WARN("fail to add server", K(ret)); } } @@ -390,7 +385,7 @@ int ObRootMinorFreeze::init_params_by_server(const ObIArray &server_list if (server_list.count() > 0) { for (int i = 0; i < server_list.count() && OB_SUCC(ret); ++i) { if (is_server_alive(server_list.at(i))) { - if (OB_FAIL(params.add_server(server_list.at(i)))) { + if (OB_FAIL(params.push_back_param(server_list.at(i)))) { LOG_WARN("fail to add server, ", K(ret)); } } else { @@ -407,7 +402,7 @@ int ObRootMinorFreeze::init_params_by_server(const ObIArray &server_list LOG_WARN("fail to get alive servers, ", K(ret)); } else { for (int i = 0; i < target_server_list.count() && OB_SUCC(ret); ++i) { - if (OB_FAIL(params.add_server(target_server_list.at(i)))) { + if (OB_FAIL(params.push_back_param(target_server_list.at(i)))) { LOG_WARN("fail to add server, ", K(ret)); } } @@ -417,32 +412,19 @@ int ObRootMinorFreeze::init_params_by_server(const ObIArray &server_list return ret; } -int ObRootMinorFreeze::ParamsContainer::add_server(const ObAddr &server) -{ - int ret = OB_SUCCESS; - - MinorFreezeParam param; - param.server = server; - // leave empty with param.arg means **all** - - if (OB_FAIL(params_.push_back(param))) { - LOG_WARN("fail to push server, ", K(ret)); - } - - return ret; -} - -int ObRootMinorFreeze::ParamsContainer::add_tenant_server(uint64_t tenant_id, - const common::ObTabletID &tablet_id, - const ObAddr &server) +int ObRootMinorFreeze::ParamsContainer::push_back_param(const common::ObAddr &server, + const uint64_t tenant_id, + share::ObLSID ls_id, + const common::ObTabletID &tablet_id) { int ret = OB_SUCCESS; MinorFreezeParam param; param.server = server; + param.arg.ls_id_ = ls_id; param.arg.tablet_id_ = tablet_id; - if (OB_FAIL(param.arg.tenant_ids_.push_back(tenant_id))) { + if (0 != tenant_id && OB_FAIL(param.arg.tenant_ids_.push_back(tenant_id))) { LOG_WARN("fail to push tenant_id, ", K(ret)); } else if (OB_FAIL(params_.push_back(param))) { LOG_WARN("fail to push tenant_id & server, ", K(ret)); diff --git a/src/rootserver/ob_root_minor_freeze.h b/src/rootserver/ob_root_minor_freeze.h index eedd38728..c05360f29 100644 --- a/src/rootserver/ob_root_minor_freeze.h +++ b/src/rootserver/ob_root_minor_freeze.h @@ -46,11 +46,7 @@ public: void start(); void stop(); int destroy(); - - int try_minor_freeze(const common::ObIArray &tenant_ids, - const common::ObIArray &server_list, - const common::ObZone &zone, - const common::ObTabletID &tablet_id) const; + int try_minor_freeze(const obrpc::ObRootMinorFreezeArg &arg) const; private: typedef struct MinorFreezeParam { @@ -67,10 +63,10 @@ private: bool is_empty() const { return params_.count() <= 0; } const common::ObIArray &get_params() const { return params_; } - int add_server(const common::ObAddr &server); - int add_tenant_server(const uint64_t tenant_id, - const common::ObTabletID &tablet_id, - const common::ObAddr &server); + int push_back_param(const common::ObAddr &server, + const uint64_t tenant_id = 0, + share::ObLSID ls_id = share::INVALID_LS, + const common::ObTabletID &tablet_id = ObTabletID(ObTabletID::INVALID_TABLET_ID)); TO_STRING_KV(K_(params)); private: @@ -84,6 +80,10 @@ private: const common::ObZone &zone, bool &server_in_zone) const; + int init_params_by_ls_or_tablet(const uint64_t tenant_id, + share::ObLSID ls_id, + const common::ObTabletID &tablet_id, + ParamsContainer ¶ms) const; int init_params_by_tenant(const common::ObIArray &tenant_ids, const common::ObZone &zone, const common::ObIArray &server_list, @@ -95,10 +95,6 @@ private: int init_params_by_server(const common::ObIArray &server_list, ParamsContainer ¶ms) const; - int init_params_by_tablet_id(const uint64_t tenant_id, - const common::ObTabletID &tablet_id, - ParamsContainer ¶ms) const; - int do_minor_freeze(const ParamsContainer ¶ms) const; int check_cancel() const; diff --git a/src/rootserver/ob_root_service.cpp b/src/rootserver/ob_root_service.cpp index 70afaa351..1446f5038 100644 --- a/src/rootserver/ob_root_service.cpp +++ b/src/rootserver/ob_root_service.cpp @@ -1491,22 +1491,6 @@ int ObRootService::submit_create_inner_schema_task() return ret; } -int ObRootService::submit_async_minor_freeze_task(const ObRootMinorFreezeArg &arg) -{ - int ret = OB_SUCCESS; - ObMinorFreezeTask task(arg); - task.set_retry_times(0); //not repeat - if (!inited_) { - ret = OB_NOT_INIT; - LOG_WARN("not init", K(ret)); - } else if (OB_FAIL(task_queue_.add_async_task(task))) { - LOG_WARN("submit async minor freeze task fail", K(ret)); - } else { - LOG_INFO("submit async minor freeze task success", K(ret)); - } - return ret; -} - int ObRootService::schedule_check_server_timer_task() { int ret = OB_SUCCESS; @@ -4580,10 +4564,7 @@ int ObRootService::root_minor_freeze(const ObRootMinorFreezeArg &arg) } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); - } else if (OB_FAIL(root_minor_freeze_.try_minor_freeze(arg.tenant_ids_, - arg.server_list_, - arg.zone_, - arg.tablet_id_))) { + } else if (OB_FAIL(root_minor_freeze_.try_minor_freeze(arg))) { LOG_WARN("minor freeze failed", K(ret), K(arg)); } ROOTSERVICE_EVENT_ADD("root_service", "root_minor_freeze", K(ret), K(arg)); diff --git a/src/rootserver/ob_root_service.h b/src/rootserver/ob_root_service.h index 8bd049685..50238c4a8 100644 --- a/src/rootserver/ob_root_service.h +++ b/src/rootserver/ob_root_service.h @@ -703,7 +703,6 @@ public: int submit_upgrade_task(const obrpc::ObUpgradeJobArg &arg); int submit_upgrade_storage_format_version_task(); int submit_create_inner_schema_task(); - int submit_async_minor_freeze_task(const obrpc::ObRootMinorFreezeArg &arg); int submit_update_all_server_config_task(); int submit_max_availability_mode_task(const common::ObProtectionLevel level, const int64_t cluster_version); diff --git a/src/share/inner_table/ob_inner_table_schema.401_450.cpp b/src/share/inner_table/ob_inner_table_schema.401_450.cpp index 51555c388..92703237f 100644 --- a/src/share/inner_table/ob_inner_table_schema.401_450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.401_450.cpp @@ -3155,6 +3155,157 @@ int ObInnerTableSchema::all_tenant_rewrite_rules_schema(ObTableSchema &table_sch return ret; } +int ObInnerTableSchema::all_reserved_snapshot_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RESERVED_SNAPSHOT_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(4); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RESERVED_SNAPSHOT_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //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("snapshot_type", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("svr_ip", //column_name + ++column_id, //column_id + 3, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + MAX_IP_ADDR_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("svr_port", //column_name + ++column_id, //column_id + 4, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("create_time", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("snapshot_version", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RESERVED_SNAPSHOT_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_cluster_event_history_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp b/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp index 1f7074360..8d769416d 100644 --- a/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp @@ -1914,6 +1914,141 @@ int ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_meta_schema(ObTableSche return ret; } +int ObInnerTableSchema::all_reserved_snapshot_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_RESERVED_SNAPSHOT_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_cluster_event_history_aux_lob_meta_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp b/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp index 415aa9620..eeb2ddeef 100644 --- a/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp @@ -1284,6 +1284,96 @@ int ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_piece_schema(ObTableSch return ret; } +int ObInnerTableSchema::all_reserved_snapshot_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_RESERVED_SNAPSHOT_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_cluster_event_history_aux_lob_piece_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.h b/src/share/inner_table/ob_inner_table_schema.h index f851ed94b..0483d2a5e 100644 --- a/src/share/inner_table/ob_inner_table_schema.h +++ b/src/share/inner_table/ob_inner_table_schema.h @@ -499,6 +499,7 @@ public: static int all_spatial_reference_systems_schema(share::schema::ObTableSchema &table_schema); static int all_column_checksum_error_info_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_rewrite_rules_schema(share::schema::ObTableSchema &table_schema); + static int all_reserved_snapshot_schema(share::schema::ObTableSchema &table_schema); static int all_cluster_event_history_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_all_table_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_table_column_schema(share::schema::ObTableSchema &table_schema); @@ -1866,6 +1867,7 @@ public: static int all_spatial_reference_systems_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_column_checksum_error_info_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_rewrite_rules_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_reserved_snapshot_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_cluster_event_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_table_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_column_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); @@ -2084,6 +2086,7 @@ public: static int all_spatial_reference_systems_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_column_checksum_error_info_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_rewrite_rules_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_reserved_snapshot_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_cluster_event_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_plan_cache_stat_all_virtual_plan_cache_stat_i1_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_session_event_all_virtual_session_event_i1_schema(share::schema::ObTableSchema &table_schema); @@ -2421,6 +2424,7 @@ const schema_create_func sys_table_schema_creators [] = { ObInnerTableSchema::all_spatial_reference_systems_schema, ObInnerTableSchema::all_column_checksum_error_info_schema, ObInnerTableSchema::all_tenant_rewrite_rules_schema, + ObInnerTableSchema::all_reserved_snapshot_schema, ObInnerTableSchema::all_cluster_event_history_schema, NULL,}; @@ -3895,6 +3899,7 @@ const uint64_t tenant_space_tables [] = { OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TID, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TID, OB_ALL_TENANT_REWRITE_RULES_TID, + OB_ALL_RESERVED_SNAPSHOT_TID, OB_TENANT_VIRTUAL_ALL_TABLE_TID, OB_TENANT_VIRTUAL_TABLE_COLUMN_TID, OB_TENANT_VIRTUAL_TABLE_INDEX_TID, @@ -5074,6 +5079,7 @@ const uint64_t tenant_space_tables [] = { OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TID, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TID, OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID, + OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID, OB_ALL_TABLE_AUX_LOB_PIECE_TID, OB_ALL_COLUMN_AUX_LOB_PIECE_TID, OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TID, @@ -5271,7 +5277,8 @@ const uint64_t tenant_space_tables [] = { OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TID, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TID, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TID, - OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID, }; + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID, + OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID, }; const uint64_t all_ora_mapping_virtual_table_org_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_TID, @@ -5698,6 +5705,7 @@ const char* const tenant_space_table_names [] = { OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TNAME, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TNAME, OB_ALL_TENANT_REWRITE_RULES_TNAME, + OB_ALL_RESERVED_SNAPSHOT_TNAME, OB_TENANT_VIRTUAL_ALL_TABLE_TNAME, OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME, OB_TENANT_VIRTUAL_TABLE_INDEX_TNAME, @@ -6877,6 +6885,7 @@ const char* const tenant_space_table_names [] = { OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TNAME, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TNAME, OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TNAME, + OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TNAME, OB_ALL_TABLE_AUX_LOB_PIECE_TNAME, OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME, OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TNAME, @@ -7074,7 +7083,8 @@ const char* const tenant_space_table_names [] = { OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TNAME, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TNAME, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TNAME, - OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TNAME, }; + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TNAME, + OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TNAME, }; const uint64_t only_rs_vtables [] = { OB_ALL_VIRTUAL_CORE_META_TABLE_TID, @@ -9211,6 +9221,14 @@ LOBMapping const lob_aux_table_mappings [] = { ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_piece_schema }, + { + OB_ALL_RESERVED_SNAPSHOT_TID, + OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID, + OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_reserved_snapshot_aux_lob_meta_schema, + ObInnerTableSchema::all_reserved_snapshot_aux_lob_piece_schema + }, + { OB_ALL_CLUSTER_EVENT_HISTORY_TID, OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_META_TID, @@ -9256,12 +9274,12 @@ static inline int get_sys_table_lob_aux_schema(const uint64_t tid, } const int64_t OB_CORE_TABLE_COUNT = 4; -const int64_t OB_SYS_TABLE_COUNT = 215; -const int64_t OB_VIRTUAL_TABLE_COUNT = 554; +const int64_t OB_SYS_TABLE_COUNT = 216; +const int64_t OB_VIRTUAL_TABLE_COUNT = 555; const int64_t OB_SYS_VIEW_COUNT = 611; -const int64_t OB_SYS_TENANT_TABLE_COUNT = 1385; +const int64_t OB_SYS_TENANT_TABLE_COUNT = 1387; const int64_t OB_CORE_SCHEMA_VERSION = 1; -const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1388; +const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1390; } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.lob.cpp b/src/share/inner_table/ob_inner_table_schema.lob.cpp index 6f71df9fd..1cd0c8c10 100644 --- a/src/share/inner_table/ob_inner_table_schema.lob.cpp +++ b/src/share/inner_table/ob_inner_table_schema.lob.cpp @@ -21,7 +21,7 @@ inner_lob_map_t inner_lob_map; bool lob_mapping_init() { int ret = OB_SUCCESS; - if (OB_FAIL(inner_lob_map.create(218, ObModIds::OB_INNER_LOB_HASH_SET))) { + if (OB_FAIL(inner_lob_map.create(219, ObModIds::OB_INNER_LOB_HASH_SET))) { SERVER_LOG(WARN, "fail to create inner lob map", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(lob_aux_table_mappings); ++i) { diff --git a/src/share/inner_table/ob_inner_table_schema_constants.h b/src/share/inner_table/ob_inner_table_schema_constants.h index a10c77e22..169d4176b 100644 --- a/src/share/inner_table/ob_inner_table_schema_constants.h +++ b/src/share/inner_table/ob_inner_table_schema_constants.h @@ -245,6 +245,7 @@ const uint64_t OB_ALL_SERVICE_EPOCH_TID = 412; // "__all_service_epoch" const uint64_t OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TID = 413; // "__all_spatial_reference_systems" const uint64_t OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TID = 416; // "__all_column_checksum_error_info" const uint64_t OB_ALL_TENANT_REWRITE_RULES_TID = 443; // "__all_tenant_rewrite_rules" +const uint64_t OB_ALL_RESERVED_SNAPSHOT_TID = 444; // "__all_reserved_snapshot" const uint64_t OB_ALL_CLUSTER_EVENT_HISTORY_TID = 445; // "__all_cluster_event_history" const uint64_t OB_TENANT_VIRTUAL_ALL_TABLE_TID = 10001; // "__tenant_virtual_all_table" const uint64_t OB_TENANT_VIRTUAL_TABLE_COLUMN_TID = 10002; // "__tenant_virtual_table_column" @@ -1612,6 +1613,7 @@ const uint64_t OB_ALL_SERVICE_EPOCH_AUX_LOB_META_TID = 50412; // "__all_service_ const uint64_t OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TID = 50413; // "__all_spatial_reference_systems_aux_lob_meta" const uint64_t OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TID = 50416; // "__all_column_checksum_error_info_aux_lob_meta" const uint64_t OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID = 50443; // "__all_tenant_rewrite_rules_aux_lob_meta" +const uint64_t OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID = 50444; // "__all_reserved_snapshot_aux_lob_meta" const uint64_t OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_META_TID = 50445; // "__all_cluster_event_history_aux_lob_meta" const uint64_t OB_ALL_TABLE_AUX_LOB_PIECE_TID = 60003; // "__all_table_aux_lob_piece" const uint64_t OB_ALL_COLUMN_AUX_LOB_PIECE_TID = 60004; // "__all_column_aux_lob_piece" @@ -1830,6 +1832,7 @@ const uint64_t OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TID = 60412; // "__all_service const uint64_t OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TID = 60413; // "__all_spatial_reference_systems_aux_lob_piece" const uint64_t OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TID = 60416; // "__all_column_checksum_error_info_aux_lob_piece" const uint64_t OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID = 60443; // "__all_tenant_rewrite_rules_aux_lob_piece" +const uint64_t OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID = 60444; // "__all_reserved_snapshot_aux_lob_piece" const uint64_t OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_PIECE_TID = 60445; // "__all_cluster_event_history_aux_lob_piece" const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TID = 14999; // "__all_virtual_plan_cache_stat" const uint64_t OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TID = 14998; // "__all_virtual_session_event" @@ -2154,6 +2157,7 @@ const char *const OB_ALL_SERVICE_EPOCH_TNAME = "__all_service_epoch"; const char *const OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TNAME = "__all_spatial_reference_systems"; const char *const OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TNAME = "__all_column_checksum_error_info"; const char *const OB_ALL_TENANT_REWRITE_RULES_TNAME = "__all_tenant_rewrite_rules"; +const char *const OB_ALL_RESERVED_SNAPSHOT_TNAME = "__all_reserved_snapshot"; const char *const OB_ALL_CLUSTER_EVENT_HISTORY_TNAME = "__all_cluster_event_history"; const char *const OB_TENANT_VIRTUAL_ALL_TABLE_TNAME = "__tenant_virtual_all_table"; const char *const OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME = "__tenant_virtual_table_column"; @@ -3521,6 +3525,7 @@ const char *const OB_ALL_SERVICE_EPOCH_AUX_LOB_META_TNAME = "__all_service_epoch const char *const OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TNAME = "__all_spatial_reference_systems_aux_lob_meta"; const char *const OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TNAME = "__all_column_checksum_error_info_aux_lob_meta"; const char *const OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TNAME = "__all_tenant_rewrite_rules_aux_lob_meta"; +const char *const OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TNAME = "__all_reserved_snapshot_aux_lob_meta"; const char *const OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_META_TNAME = "__all_cluster_event_history_aux_lob_meta"; const char *const OB_ALL_TABLE_AUX_LOB_PIECE_TNAME = "__all_table_aux_lob_piece"; const char *const OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME = "__all_column_aux_lob_piece"; @@ -3739,6 +3744,7 @@ const char *const OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TNAME = "__all_service_epoc const char *const OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TNAME = "__all_spatial_reference_systems_aux_lob_piece"; const char *const OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TNAME = "__all_column_checksum_error_info_aux_lob_piece"; const char *const OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TNAME = "__all_tenant_rewrite_rules_aux_lob_piece"; +const char *const OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TNAME = "__all_reserved_snapshot_aux_lob_piece"; const char *const OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_PIECE_TNAME = "__all_cluster_event_history_aux_lob_piece"; const char *const OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TNAME = "__idx_11003_all_virtual_plan_cache_stat_i1"; const char *const OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TNAME = "__idx_11013_all_virtual_session_event_i1"; 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 cef98948d..5288513d2 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -5191,7 +5191,29 @@ def_table_schema( ], ) -# 444 : __all_reserved_snapshot +def_table_schema( + owner = 'handora.qc', + table_name = '__all_reserved_snapshot', + table_id = '444', + table_type = 'SYSTEM_TABLE', + gm_columns = [], + rowkey_columns = [ + ('tenant_id', 'int'), + ('snapshot_type', 'uint'), + ('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'), + ('svr_port', 'uint'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + + normal_columns = [ + ('create_time', 'uint'), + ('snapshot_version', 'uint'), + ('status', 'uint'), + ], +) + def_table_schema( owner = 'yanmu.ztl', table_name = '__all_cluster_event_history', diff --git a/src/share/inner_table/ob_inner_table_schema_misc.ipp b/src/share/inner_table/ob_inner_table_schema_misc.ipp index 3a88b1f01..4bfbd3cfe 100644 --- a/src/share/inner_table/ob_inner_table_schema_misc.ipp +++ b/src/share/inner_table/ob_inner_table_schema_misc.ipp @@ -3387,6 +3387,9 @@ case OB_ALL_LS_STATUS_AUX_LOB_PIECE_TID: case OB_ALL_MERGE_INFO_TID: case OB_ALL_MERGE_INFO_AUX_LOB_META_TID: case OB_ALL_MERGE_INFO_AUX_LOB_PIECE_TID: +case OB_ALL_RESERVED_SNAPSHOT_TID: +case OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID: +case OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID: case OB_ALL_RESTORE_INFO_TID: case OB_ALL_RESTORE_INFO_AUX_LOB_META_TID: case OB_ALL_RESTORE_INFO_AUX_LOB_PIECE_TID: diff --git a/src/share/ob_all_server_tracer.cpp b/src/share/ob_all_server_tracer.cpp index 8ff12e71a..5d3c9fc31 100644 --- a/src/share/ob_all_server_tracer.cpp +++ b/src/share/ob_all_server_tracer.cpp @@ -50,7 +50,7 @@ int ObServerTraceMap::init() int ObServerTraceMap::is_server_exist(const common::ObAddr &server, bool &exist) const { - int ret = OB_NOT_SUPPORTED; + int ret = OB_SUCCESS; if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("server trace map has not inited", KR(ret)); @@ -76,23 +76,53 @@ int ObServerTraceMap::is_server_exist(const common::ObAddr &server, bool &exist) int ObServerTraceMap::check_server_alive(const ObAddr &server, bool &is_alive) const { - int ret = OB_NOT_SUPPORTED; - UNUSED(server); - UNUSED(is_alive); + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("server trace map has not inited", KR(ret)); + } else if (!server.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid server", K(server), K(ret)); + } else { + SpinRLockGuard guard(lock_); + ObServerStatus status; + if (OB_FAIL(find_server_status(server, status))) { + LOG_WARN("fail to find server status", K(server), K_(server_status_arr), KR(ret)); + } else { + is_alive = status.is_alive(); + } + } + return ret; } -int ObServerTraceMap::check_in_service(const ObAddr &addr, bool &service_started) const +int ObServerTraceMap::check_in_service(const ObAddr &server, bool &service_started) const { - int ret = OB_NOT_SUPPORTED; - UNUSED(addr); - UNUSED(service_started); + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("server trace map has not inited", KR(ret)); + } else if (!server.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid server", K(server), K(ret)); + } else { + SpinRLockGuard guard(lock_); + ObServerStatus status; + if (OB_FAIL(find_server_status(server, status))) { + LOG_WARN("fail to find server status", K(server), K_(server_status_arr), KR(ret)); + } else { + service_started = status.in_service(); + } + } + return ret; } int ObServerTraceMap::check_server_permanent_offline(const ObAddr &addr, bool &is_offline) const { - int ret = OB_NOT_SUPPORTED; + int ret = OB_SUCCESS; SpinRLockGuard guard(lock_); ObServerStatus status; @@ -108,7 +138,32 @@ int ObServerTraceMap::check_server_permanent_offline(const ObAddr &addr, bool &i if ((OB_ENTRY_NOT_EXIST == ret) && server_status_arr_.empty()) { // if server list is empty, treat as not offline ret = OB_SUCCESS; - is_offline = false; + is_offline = false; + } + + return ret; +} + +int ObServerTraceMap::for_each_server_status( + const ObFunction &functor) +{ + int ret = OB_SUCCESS; + SpinRLockGuard guard(lock_); + + ObServerStatus status; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < server_status_arr_.count(); i++) { + ObServerStatus &status = server_status_arr_[i]; + if (OB_UNLIKELY(!functor.is_valid())) { + ret = OB_EAGAIN; + LOG_WARN("functor is invalid"); + } else if (OB_FAIL(functor(status))) { + LOG_WARN("invoke functor failed", K(ret), K(status)); + } + } } return ret; @@ -268,3 +323,9 @@ int ObAllServerTracer::check_migrate_in_blocked(const ObAddr &addr, bool &is_blo { return trace_map_.check_migrate_in_blocked(addr, is_block); } + +int ObAllServerTracer::for_each_server_status( + const ObFunction &functor) +{ + return trace_map_.for_each_server_status(functor); +} diff --git a/src/share/ob_all_server_tracer.h b/src/share/ob_all_server_tracer.h index 747eb314b..9b1aea0d1 100644 --- a/src/share/ob_all_server_tracer.h +++ b/src/share/ob_all_server_tracer.h @@ -17,6 +17,7 @@ #include "lib/hash/ob_hashset.h" #include "share/ob_iserver_trace.h" #include "share/ob_server_table_operator.h" +#include "lib/function/ob_function.h" namespace oceanbase { @@ -36,6 +37,7 @@ public: virtual int check_server_permanent_offline(const common::ObAddr &server, bool &is_offline) const; virtual int is_server_stopped(const common::ObAddr &server, bool &is_stopped) const; int refresh(); + int for_each_server_status(const ObFunction &functor); private: int find_server_status(const ObAddr &addr, ObServerStatus &status) const; @@ -67,6 +69,7 @@ class ObAllServerTracer : public share::ObIServerTrace public: static ObAllServerTracer &get_instance(); int init(int tg_id, ObServerTraceTask &trace_task); + int for_each_server_status(const ObFunction &functor); virtual int is_server_exist(const common::ObAddr &server, bool &exist) const; virtual int check_server_alive(const common::ObAddr &server, bool &is_alive) const; virtual int check_in_service(const common::ObAddr &addr, bool &service_started) const; diff --git a/src/share/ob_debug_sync_point.h b/src/share/ob_debug_sync_point.h index 446835adf..dd9138507 100644 --- a/src/share/ob_debug_sync_point.h +++ b/src/share/ob_debug_sync_point.h @@ -144,6 +144,9 @@ class ObString; ACT(BEFORE_RESTORE_PARTITIONS,)\ ACT(BEFORE_BATCH_PROCESS_TASK,)\ ACT(BEFORE_INSERT_ROWS,)\ + ACT(AFTER_INSERT_ROWS,)\ + ACT(AFTER_INSERT_ROW,)\ + ACT(AFTER_TABLE_SCAN,)\ ACT(BEFORE_BUILD_LOCAL_INDEX_REFRESH_TABLES,)\ ACT(BEFORE_BUILD_LOCAL_INDEX_REFRESH_TABLES_MID,)\ ACT(BEFORE_SLOG_UPDATE_FLUSH_CURSOR,)\ diff --git a/src/share/ob_rpc_struct.cpp b/src/share/ob_rpc_struct.cpp index d2470e46c..8c4acd8d2 100644 --- a/src/share/ob_rpc_struct.cpp +++ b/src/share/ob_rpc_struct.cpp @@ -4158,7 +4158,8 @@ OB_SERIALIZE_MEMBER(ObRootMajorFreezeArg, OB_SERIALIZE_MEMBER(ObMinorFreezeArg, tenant_ids_, - tablet_id_); + tablet_id_, + ls_id_); int ObMinorFreezeArg::assign(const ObMinorFreezeArg &other) { @@ -4167,6 +4168,7 @@ int ObMinorFreezeArg::assign(const ObMinorFreezeArg &other) LOG_WARN("assign tenant_ids_ failed", K(ret), K(other.tenant_ids_)); } else { tablet_id_ = other.tablet_id_; + ls_id_ = other.ls_id_; } return ret; } @@ -4175,7 +4177,8 @@ OB_SERIALIZE_MEMBER(ObRootMinorFreezeArg, tenant_ids_, server_list_, zone_, - tablet_id_); + tablet_id_, + ls_id_); int ObRootMinorFreezeArg::assign(const ObRootMinorFreezeArg &other) { @@ -4188,6 +4191,7 @@ int ObRootMinorFreezeArg::assign(const ObRootMinorFreezeArg &other) LOG_WARN("assign zone_ failed", K(ret), K(other.zone_)); } else { tablet_id_ = other.tablet_id_; + ls_id_ = other.ls_id_; } return ret; } diff --git a/src/share/ob_rpc_struct.h b/src/share/ob_rpc_struct.h index 6e228152c..11989bd97 100644 --- a/src/share/ob_rpc_struct.h +++ b/src/share/ob_rpc_struct.h @@ -4898,6 +4898,7 @@ public: { tenant_ids_.reset(); tablet_id_.reset(); + ls_id_.reset(); } bool is_valid() const @@ -4905,10 +4906,11 @@ public: return true; } - TO_STRING_KV(K_(tenant_ids), K_(tablet_id)); + TO_STRING_KV(K_(tenant_ids), K_(ls_id), K_(tablet_id)); common::ObSArray tenant_ids_; common::ObTabletID tablet_id_; + share::ObLSID ls_id_; }; struct ObRootMinorFreezeArg @@ -4924,6 +4926,7 @@ public: server_list_.reset(); zone_.reset(); tablet_id_.reset(); + ls_id_.reset(); } bool is_valid() const @@ -4931,12 +4934,13 @@ public: return true; } - TO_STRING_KV(K_(tenant_ids), K_(server_list), K_(zone), K_(tablet_id)); + TO_STRING_KV(K_(tenant_ids), K_(server_list), K_(zone), K_(tablet_id), K_(ls_id)); common::ObSArray tenant_ids_; common::ObSArray server_list_; common::ObZone zone_; common::ObTabletID tablet_id_; + share::ObLSID ls_id_; }; struct ObSyncPGPartitionMTFinishArg diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index f66d61e86..760827a33 100644 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -305,7 +305,10 @@ DEF_INT(px_workers_per_cpu_quota, OB_CLUSTER_PARAMETER, "10", "[0,20]", "the maximum number of threads that can be scheduled concurrently. Range: [0, 20]", ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); DEF_INT(undo_retention, OB_TENANT_PARAMETER, "1800", "[0, 4294967295]", - "the low threshold value of undo retention. The system retains undo for at least the time specified in this config. Range: [0, 4294967295]", + "the low threshold value of undo retention. The system retains undo for at least the time specified in this config when active txn protection is banned. Range: [0, 4294967295]", + ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); +DEF_BOOL(_mvcc_gc_using_min_txn_snapshot, OB_TENANT_PARAMETER, "True", + "specifies enable mvcc gc using active txn snapshot", ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); DEF_BOOL(_rowsets_enabled, OB_TENANT_PARAMETER, "True", "specifies whether vectorized sql execution engine is activated", @@ -656,6 +659,10 @@ DEF_TIME(_ob_trans_rpc_timeout, OB_CLUSTER_PARAMETER, "3s", "[0s, 3600s]", DEF_BOOL(enable_early_lock_release, OB_TENANT_PARAMETER, "True", "enable early lock release", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); +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_TIME(_ob_get_gts_ahead_interval, OB_CLUSTER_PARAMETER, "0s", "[0s, 1s]", "get gts ahead interval. Range: [0s, 1s]", @@ -780,7 +787,7 @@ DEF_BOOL(_enable_compaction_diagnose, OB_CLUSTER_PARAMETER, "False", DEF_STR(_force_skip_encoding_partition_id, OB_CLUSTER_PARAMETER, "", "force the specified partition to major without encoding row store, only for emergency!", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); -DEF_CAP(_private_buffer_size, OB_CLUSTER_PARAMETER, "256K", "[0B,)" +DEF_CAP(_private_buffer_size, OB_CLUSTER_PARAMETER, "16K", "[0B,)" "the trigger remaining data size within transaction for immediate logging, 0B represents not trigger immediate logging" "Range: [0B, total size of memory]", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); diff --git a/src/share/rc/ob_tenant_base.h b/src/share/rc/ob_tenant_base.h index 3c326605b..c77bd4e0f 100644 --- a/src/share/rc/ob_tenant_base.h +++ b/src/share/rc/ob_tenant_base.h @@ -80,6 +80,9 @@ namespace transaction { class ObTableLockService; } } +namespace concurrency_control { + class ObMultiVersionGarbageCollector; // MVCC GC +} namespace logservice { @@ -210,6 +213,7 @@ namespace detector storage::ObTenantFreezeInfoMgr*, \ transaction::ObTxLoopWorker *, \ storage::ObAccessService*, \ + concurrency_control::ObMultiVersionGarbageCollector*, \ sql::ObUDRMgr*, \ ObTestModule* \ ) diff --git a/src/sql/engine/cmd/ob_alter_system_executor.cpp b/src/sql/engine/cmd/ob_alter_system_executor.cpp index 81fceffcb..bbab1a446 100644 --- a/src/sql/engine/cmd/ob_alter_system_executor.cpp +++ b/src/sql/engine/cmd/ob_alter_system_executor.cpp @@ -74,6 +74,7 @@ int ObFreezeExecutor::execute(ObExecContext &ctx, ObFreezeStmt &stmt) } else { arg.zone_ = stmt.get_zone(); arg.tablet_id_ = stmt.get_tablet_id(); + arg.ls_id_ = stmt.get_ls_id(); } if (OB_SUCC(ret)) { // get all tenants to freeze diff --git a/src/sql/ob_sql_trans_control.cpp b/src/sql/ob_sql_trans_control.cpp index 70cf3582a..65d0018b0 100644 --- a/src/sql/ob_sql_trans_control.cpp +++ b/src/sql/ob_sql_trans_control.cpp @@ -586,7 +586,10 @@ int ObSqlTransControl::stmt_setup_snapshot_(ObSQLSessionInfo *session, } else if (plan->is_plain_insert() && session->get_tx_isolation() != ObTxIsolationLevel::SERIAL && session->get_tx_isolation() != ObTxIsolationLevel::RR) { + auto &tx_desc = *session->get_tx_desc(); snapshot.init_none_read(); + snapshot.core_.tx_id_ = tx_desc.get_tx_id(); + snapshot.core_.scn_ = common::ObSequence::get_max_seq_no(); } else { auto &tx_desc = *session->get_tx_desc(); int64_t stmt_expire_ts = get_stmt_expire_ts(plan_ctx, *session); diff --git a/src/sql/optimizer/ob_optimizer.cpp b/src/sql/optimizer/ob_optimizer.cpp index 2380fec8c..ae32c46d0 100644 --- a/src/sql/optimizer/ob_optimizer.cpp +++ b/src/sql/optimizer/ob_optimizer.cpp @@ -540,9 +540,6 @@ int ObOptimizer::check_pdml_supported_feature(const ObDMLStmt &stmt, session.get_effective_tenant_id(), main_table_tid, with_unique_local_idx))) { LOG_WARN("fail check if table with local unqiue index", K(main_table_tid), K(ret)); - } else if (with_unique_local_idx) { - is_use_pdml = false; - ctx_.add_plan_note(PDML_DISABLED_BY_LOCAL_UK); } else if (stmt::T_UPDATE == stmt.get_stmt_type()) { for (int i = 0; OB_SUCC(ret) && is_use_pdml && i < table_infos.at(0)->column_exprs_.count(); i++) { diff --git a/src/sql/parser/sql_parser_mysql_mode.y b/src/sql/parser/sql_parser_mysql_mode.y index 52ae511d1..3f42a5ebb 100755 --- a/src/sql/parser/sql_parser_mysql_mode.y +++ b/src/sql/parser/sql_parser_mysql_mode.y @@ -441,7 +441,7 @@ END_P SET_VAR DELIMITER %type create_view_stmt view_name opt_column_list opt_table_id opt_tablet_id view_select_stmt opt_check_option %type name_list %type partition_role ls_role zone_desc opt_zone_desc server_or_zone opt_server_or_zone opt_partitions opt_subpartitions add_or_alter_zone_options alter_or_change_or_modify -%type ls opt_tenant_list_and_tablet_id ls_server_or_server_or_zone_or_tenant add_or_alter_zone_option +%type ls opt_tenant_list_or_ls_or_tablet_id ls_server_or_server_or_zone_or_tenant add_or_alter_zone_option %type opt_tenant_list_v2 %type suspend_or_resume tenant_name opt_tenant_name cache_name opt_cache_name file_id opt_file_id cancel_task_type %type sql_id_expr opt_sql_id @@ -13854,7 +13854,7 @@ ALTER SYSTEM CHECKPOINT malloc_non_terminal_node($$, result->malloc_pool_, T_FREEZE, 2, type, NULL); } | -ALTER SYSTEM MINOR FREEZE opt_tenant_list_and_tablet_id opt_server_list opt_zone_desc +ALTER SYSTEM MINOR FREEZE opt_tenant_list_or_ls_or_tablet_id opt_server_list opt_zone_desc { ParseNode *type = NULL; malloc_terminal_node(type, result->malloc_pool_, T_INT); @@ -15098,17 +15098,22 @@ LS opt_equal_mark INTNUM } ; -opt_tenant_list_and_tablet_id: +opt_tenant_list_or_ls_or_tablet_id: tenant_list_tuple opt_tablet_id { malloc_non_terminal_node($$, result->malloc_pool_, T_TENANT_TABLET, 2, $1, $2); } +| tenant_list_tuple ls opt_tablet_id +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_TENANT_LS_TABLET, 3, $1, $2, $3); +} | /*EMPTY*/ { $$ = NULL; } ; + ls_server_or_server_or_zone_or_tenant: ls ip_port tenant_name { diff --git a/src/sql/resolver/cmd/ob_alter_system_resolver.cpp b/src/sql/resolver/cmd/ob_alter_system_resolver.cpp index d51ea7513..bda637351 100644 --- a/src/sql/resolver/cmd/ob_alter_system_resolver.cpp +++ b/src/sql/resolver/cmd/ob_alter_system_resolver.cpp @@ -270,11 +270,29 @@ int ObAlterSystemResolverUtil::resolve_ls_id(const ParseNode *parse_tree, int64_ { int ret = OB_SUCCESS; if (NULL == parse_tree) { + ret = OB_ERR_NULL_VALUE; LOG_WARN("node should not be null"); } else if (OB_FAIL(sanity_check(parse_tree, T_LS))) { LOG_WARN("sanity check failed"); } else { ls_id = parse_tree->children_[0]->value_; + FLOG_INFO("resolve ls id", K(ls_id)); + } + return ret; +} + +int ObAlterSystemResolverUtil::resolve_tablet_id(const ParseNode *opt_tablet_id, ObTabletID &tablet_id) +{ + int ret = OB_SUCCESS; + + if (NULL == opt_tablet_id) { + ret = OB_ERR_NULL_VALUE; + LOG_WARN("opt_tablet_id should not be null"); + } else if (OB_FAIL(sanity_check(opt_tablet_id, T_TABLET_ID))) { + LOG_WARN("sanity check failed"); + } else { + tablet_id = opt_tablet_id->children_[0]->value_; + FLOG_INFO("resolve tablet_id", K(tablet_id)); } return ret; } @@ -416,32 +434,11 @@ int ObAlterSystemResolverUtil::resolve_tenant( return ret; } -// resolve tablet id -int ObAlterSystemResolverUtil::resolve_tablet_id(const ParseNode &tenants_tablet_node, ObFreezeStmt &freeze_stmt) -{ - int ret = OB_SUCCESS; - ObTabletID &tablet_id = freeze_stmt.get_tablet_id(); - - if (OB_UNLIKELY(NULL == tenants_tablet_node.children_) - || OB_UNLIKELY(0 == tenants_tablet_node.num_child_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("children of tenant should not be null", KR(ret)); - } else if (OB_NOT_NULL(tenants_tablet_node.children_[1]) - && OB_NOT_NULL(tenants_tablet_node.children_[1]->children_[0])) { - tablet_id = tenants_tablet_node.children_[1]->children_[0]->value_; - FLOG_INFO("resolve tablet_id", K(tablet_id)); - } - - return ret; -} int ObFreezeResolver::resolve(const ParseNode &parse_tree) { int ret = OB_SUCCESS; ObFreezeStmt *freeze_stmt = NULL; - ParseNode *servers_node = NULL; - ParseNode *tenants_and_tablet_node = NULL; - ParseNode *zone_node = NULL; if (OB_UNLIKELY(NULL == parse_tree.children_) || OB_UNLIKELY(parse_tree.num_child_ < 2)) { ret = OB_ERR_UNEXPECTED; @@ -460,37 +457,35 @@ int ObFreezeResolver::resolve(const ParseNode &parse_tree) ret = OB_ERR_UNEXPECTED; LOG_WARN("wrong freeze type", K(parse_tree.children_[0]->type_)); } else { - const uint64_t cur_tenant_id = session_info_->get_effective_tenant_id(); stmt_ = freeze_stmt; if (1 == parse_tree.children_[0]->value_) { // MAJOR FREEZE + freeze_stmt->set_major_freeze(true); if (OB_UNLIKELY(2 != parse_tree.num_child_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("wrong freeze parse tree", K(parse_tree.num_child_)); } else { - freeze_stmt->set_major_freeze(true); - tenants_and_tablet_node = parse_tree.children_[1]; - - if (NULL == tenants_and_tablet_node) { - // if tenants_and_tablet_node == null, add owned tenant_id - if (OB_FAIL(freeze_stmt->get_tenant_ids().push_back(cur_tenant_id))) { - LOG_WARN("fail to push owned tenant id ", KR(ret), "owned tenant_id", cur_tenant_id); - } - } else { - if (OB_SYS_TENANT_ID != cur_tenant_id) { - ret = OB_ERR_NO_PRIVILEGE; - LOG_WARN("Only sys tenant can add suffix opt(tenant=name)", KR(ret), K(cur_tenant_id)); - } + ParseNode *opt_tenant_list_v2 = parse_tree.children_[1]; + if (OB_FAIL(resolve_major_freeze_(freeze_stmt, opt_tenant_list_v2))) { + LOG_WARN("resolve major freeze failed", KR(ret), KP(opt_tenant_list_v2)); } } - } else if (2 == parse_tree.children_[0]->value_) { // MINOR FREEZE + } else if (2 == parse_tree.children_[0]->value_) { // MINOR FREEZE + freeze_stmt->set_major_freeze(false); if (OB_UNLIKELY(4 != parse_tree.num_child_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("wrong freeze parse tree", K(parse_tree.num_child_)); } else { - freeze_stmt->set_major_freeze(false); - tenants_and_tablet_node = parse_tree.children_[1]; - servers_node = parse_tree.children_[2]; - zone_node = parse_tree.children_[3]; + ParseNode *opt_tenant_list_or_ls_or_tablet_id = parse_tree.children_[1]; + ParseNode *opt_server_list = parse_tree.children_[2]; + ParseNode *opt_zone_desc = parse_tree.children_[3]; + if (OB_FAIL(resolve_minor_freeze_( + freeze_stmt, opt_tenant_list_or_ls_or_tablet_id, opt_server_list, opt_zone_desc))) { + LOG_WARN("resolve minor freeze failed", + KR(ret), + KP(opt_tenant_list_or_ls_or_tablet_id), + KP(opt_server_list), + KP(opt_zone_desc)); + } } } else { ret = OB_ERR_UNEXPECTED; @@ -498,60 +493,146 @@ int ObFreezeResolver::resolve(const ParseNode &parse_tree) } } - // resolve zone - if (OB_SUCC(ret) && (NULL != zone_node)) { - if (OB_FAIL(Util::resolve_zone(zone_node, freeze_stmt->get_zone()))) { - LOG_WARN("resolve zone failed", K(ret)); + return ret; +} + +int ObFreezeResolver::resolve_major_freeze_(ObFreezeStmt *freeze_stmt, ParseNode *opt_tenant_list_v2) +{ + int ret = OB_SUCCESS; + const uint64_t cur_tenant_id = session_info_->get_effective_tenant_id(); + + if (NULL == opt_tenant_list_v2) { + // if opt_tenant_list_v2 == NULL, add owned tenant_id + if (OB_FAIL(freeze_stmt->get_tenant_ids().push_back(cur_tenant_id))) { + LOG_WARN("fail to push owned tenant id ", KR(ret), "owned tenant_id", cur_tenant_id); } - } - - // resolve tenant and tablet id - if (OB_SUCC(ret) && (NULL != tenants_and_tablet_node)) { - const uint64_t tenant_id = session_info_->get_effective_tenant_id(); - - if ((T_TENANT_LIST == tenants_and_tablet_node->type_) - || (T_TENANT_TABLET == tenants_and_tablet_node->type_)) { - if (OB_UNLIKELY(nullptr == tenants_and_tablet_node->children_) - || OB_UNLIKELY(0 == tenants_and_tablet_node->num_child_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("children of tenant should not be null", KR(ret)); - } else { - bool affect_all = false; - if (T_TENANT_LIST == tenants_and_tablet_node->type_) { - if (OB_FAIL(Util::resolve_tenant(*tenants_and_tablet_node, tenant_id, freeze_stmt->get_tenant_ids(), affect_all))) { - LOG_WARN("fail to resolve tenant", KR(ret)); - } else if (affect_all) { - freeze_stmt->set_freeze_all(affect_all); - } - } else if (T_TENANT_TABLET == tenants_and_tablet_node->type_) { - const ParseNode *tenants_node = tenants_and_tablet_node->children_[0]; - if (OB_UNLIKELY(nullptr == tenants_node)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tenants_node is nullptr", KR(ret)); - } else if (OB_FAIL(Util::resolve_tenant(*tenants_node, tenant_id, freeze_stmt->get_tenant_ids(), affect_all))) { - LOG_WARN("fail to resolve tenant", KR(ret)); - } else if (affect_all) { - freeze_stmt->set_freeze_all(affect_all); - } - if (FAILEDx(Util::resolve_tablet_id(*tenants_and_tablet_node, *freeze_stmt))) { - LOG_WARN("fail to resolve tablet id", KR(ret)); - } - } + } else if (OB_SYS_TENANT_ID != cur_tenant_id) { + ret = OB_ERR_NO_PRIVILEGE; + LOG_WARN("Only sys tenant can add suffix opt(tenant=name)", KR(ret), K(cur_tenant_id)); + } else if ((T_TENANT_LIST == opt_tenant_list_v2->type_)) { + // if opt_tenant_list_v2 != NULL && type == T_TENANT_LIST, resolve it + if (OB_UNLIKELY(nullptr == opt_tenant_list_v2->children_) || OB_UNLIKELY(0 == opt_tenant_list_v2->num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("children of tenant should not be null", KR(ret)); + } else { + bool affect_all = false; + if (OB_FAIL(Util::resolve_tenant(*opt_tenant_list_v2, cur_tenant_id, freeze_stmt->get_tenant_ids(), affect_all))) { + LOG_WARN("fail to resolve tenant", KR(ret)); + } else if (affect_all) { + freeze_stmt->set_freeze_all(affect_all); } } + } else { + LOG_WARN("invalid type when resolve opt_tenant_list_v2", K(T_TENANT_LIST), K(opt_tenant_list_v2->type_)); } - // resolve observer list - if (OB_SUCC(ret) && (NULL != servers_node)) { - if (OB_UNLIKELY(NULL == servers_node->children_) - || OB_UNLIKELY(0 == servers_node->num_child_)) { + return ret; +} + +int ObFreezeResolver::resolve_minor_freeze_(ObFreezeStmt *freeze_stmt, + ParseNode *opt_tenant_list_or_ls_or_tablet_id, + ParseNode *opt_server_list, + ParseNode *opt_zone_desc) +{ + int ret = OB_SUCCESS; + const uint64_t cur_tenant_id = session_info_->get_effective_tenant_id(); + + if (OB_SUCC(ret) && OB_NOT_NULL(opt_tenant_list_or_ls_or_tablet_id) && + OB_FAIL(resolve_tenant_ls_tablet_(freeze_stmt, opt_tenant_list_or_ls_or_tablet_id))) { + LOG_WARN("resolve tenant ls table failed", KR(ret)); + } + + if (OB_SUCC(ret) && OB_NOT_NULL(opt_server_list) && OB_FAIL(resolve_server_list_(freeze_stmt, opt_server_list))) { + LOG_WARN("resolve server list failed", KR(ret)); + } + + if (OB_SUCC(ret) && OB_NOT_NULL(opt_zone_desc) && + OB_FAIL(Util::resolve_zone(opt_zone_desc, freeze_stmt->get_zone()))) { + LOG_WARN("resolve zone desc failed", KR(ret)); + } + + return ret; +} + +int ObFreezeResolver::resolve_tenant_ls_tablet_(ObFreezeStmt *freeze_stmt, + ParseNode *opt_tenant_list_or_ls_or_tablet_id) +{ + int ret = OB_SUCCESS; + const uint64_t cur_tenant_id = session_info_->get_effective_tenant_id(); + + if (OB_ISNULL(opt_tenant_list_or_ls_or_tablet_id->children_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("children of tenant should not be null", KR(ret)); + } else { + bool affect_all = false; + const ParseNode *tenant_list_tuple = nullptr; + const ParseNode *opt_tablet_id = nullptr; + const ParseNode *ls_id = nullptr; + + switch (opt_tenant_list_or_ls_or_tablet_id->type_) { + case T_TENANT_TABLET: + if (opt_tenant_list_or_ls_or_tablet_id->num_child_ != 2) { + LOG_WARN("invalid child num", K(opt_tenant_list_or_ls_or_tablet_id->num_child_)); + } else { + tenant_list_tuple = opt_tenant_list_or_ls_or_tablet_id->children_[0]; + opt_tablet_id = opt_tenant_list_or_ls_or_tablet_id->children_[1]; + if (OB_ISNULL(tenant_list_tuple)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant list is nullptr", KR(ret), KP(tenant_list_tuple), KP(ls_id), KP(opt_tablet_id)); + } + } + break; + case T_TENANT_LS_TABLET: + if (opt_tenant_list_or_ls_or_tablet_id->num_child_ != 3) { + LOG_WARN("invalid child num", K(opt_tenant_list_or_ls_or_tablet_id->num_child_)); + } else { + tenant_list_tuple = opt_tenant_list_or_ls_or_tablet_id->children_[0]; + ls_id = opt_tenant_list_or_ls_or_tablet_id->children_[1]; + opt_tablet_id = opt_tenant_list_or_ls_or_tablet_id->children_[2]; + if (OB_ISNULL(tenant_list_tuple) || OB_ISNULL(ls_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant_list or ls_id is nullptr", KR(ret), KP(tenant_list_tuple), KP(ls_id), KP(opt_tablet_id)); + } + } + break; + default: + LOG_WARN("invalid parse node type", + K(T_TENANT_TABLET), + K(T_TENANT_LS_TABLET), + K(opt_tenant_list_or_ls_or_tablet_id->type_)); + break; + } + + if (OB_FAIL(ret)) { + } else if (OB_NOT_NULL(tenant_list_tuple) && + OB_FAIL(Util::resolve_tenant( + *tenant_list_tuple, cur_tenant_id, freeze_stmt->get_tenant_ids(), affect_all))) { + LOG_WARN("fail to resolve tenant", KR(ret)); + } else if (OB_NOT_NULL(ls_id) && OB_FAIL(Util::resolve_ls_id(ls_id, freeze_stmt->get_ls_id()))) { + LOG_WARN("fail to resolve tablet id", KR(ret)); + } else if (OB_NOT_NULL(opt_tablet_id) && + OB_FAIL(Util::resolve_tablet_id(opt_tablet_id, freeze_stmt->get_tablet_id()))) { + LOG_WARN("fail to resolve tablet id", KR(ret)); + } else if (affect_all) { + freeze_stmt->set_freeze_all(affect_all); + } + } + + return ret; +} + +int ObFreezeResolver::resolve_server_list_(ObFreezeStmt *freeze_stmt, ParseNode *opt_server_list) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(opt_server_list->children_) + || OB_UNLIKELY(0 == opt_server_list->num_child_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("children of server_list should not be null"); } else { ObString addr_str; ObAddr server; - for (int64_t i = 0; OB_SUCC(ret) && i < servers_node->num_child_; ++i) { - ParseNode *node = servers_node->children_[i]; + for (int64_t i = 0; OB_SUCC(ret) && i < opt_server_list->num_child_; ++i) { + ParseNode *node = opt_server_list->children_[i]; if (OB_ISNULL(node)) { LOG_WARN("children of server_list should not be null"); } else { @@ -566,8 +647,6 @@ int ObFreezeResolver::resolve(const ParseNode &parse_tree) addr_str.reset(); } } - } - return ret; } diff --git a/src/sql/resolver/cmd/ob_alter_system_resolver.h b/src/sql/resolver/cmd/ob_alter_system_resolver.h index 9f4e727dd..c3f2df05f 100644 --- a/src/sql/resolver/cmd/ob_alter_system_resolver.h +++ b/src/sql/resolver/cmd/ob_alter_system_resolver.h @@ -56,7 +56,7 @@ public: static int get_tenant_ids(const ParseNode &t_node, common::ObIArray &tenant_ids); - static int resolve_tablet_id(const ParseNode &tenants_tablet_node, ObFreezeStmt &freeze_stmt); + static int resolve_tablet_id(const ParseNode *opt_tablet_id, ObTabletID &tablet_id); static int resolve_tenant(const ParseNode &tenants_node, const uint64_t tenant_id, common::ObSArray &tenant_ids, @@ -72,8 +72,6 @@ public: virtual int resolve(const ParseNode &parse_tree); \ }; -DEF_SIMPLE_CMD_RESOLVER(ObFreezeResolver); - DEF_SIMPLE_CMD_RESOLVER(ObFlushCacheResolver); DEF_SIMPLE_CMD_RESOLVER(ObFlushKVCacheResolver); @@ -189,6 +187,23 @@ private: const common::ObString &name_node, const common::ObString &value_node); }; +class ObFreezeResolver : public ObSystemCmdResolver { +public: + ObFreezeResolver(ObResolverParams ¶ms) : ObSystemCmdResolver(params) {} + virtual ~ObFreezeResolver() {} + virtual int resolve(const ParseNode &parse_tree); +private: + int resolve_major_freeze_(ObFreezeStmt *freeze_stmt, ParseNode *opt_tenant_list_v2); + int resolve_minor_freeze_(ObFreezeStmt *freeze_stmt, + ParseNode *opt_tenant_list_or_ls_or_tablet_id, + ParseNode *opt_server_list, + ParseNode *opt_zone_desc); + + int resolve_tenant_ls_tablet_(ObFreezeStmt *freeze_stmt, ParseNode *opt_tenant_list_or_ls_or_tablet_id); + int resolve_server_list_(ObFreezeStmt *freeze_stmt, ParseNode *opt_server_list); + +}; + DEF_SIMPLE_CMD_RESOLVER(ObBackupDatabaseResolver); DEF_SIMPLE_CMD_RESOLVER(ObBackupManageResolver); DEF_SIMPLE_CMD_RESOLVER(ObBackupCleanResolver); diff --git a/src/sql/resolver/cmd/ob_alter_system_stmt.h b/src/sql/resolver/cmd/ob_alter_system_stmt.h index 65c2b7154..f1671f53b 100644 --- a/src/sql/resolver/cmd/ob_alter_system_stmt.h +++ b/src/sql/resolver/cmd/ob_alter_system_stmt.h @@ -31,15 +31,15 @@ public: freeze_all_(false), opt_server_list_(), opt_tenant_ids_(), - opt_tablet_id_() - {} + opt_tablet_id_(), + opt_ls_id_(share::ObLSID::INVALID_LS_ID) {} ObFreezeStmt(common::ObIAllocator *name_pool) : ObSystemCmdStmt(name_pool, stmt::T_FREEZE), major_freeze_(false), opt_server_list_(), opt_tenant_ids_(), - opt_tablet_id_() - {} + opt_tablet_id_(), + opt_ls_id_(share::ObLSID::INVALID_LS_ID) {} virtual ~ObFreezeStmt() {} bool is_major_freeze() const { return major_freeze_; } @@ -51,12 +51,13 @@ public: inline common::ObSArray &get_tenant_ids() { return opt_tenant_ids_; } inline common::ObZone &get_zone() { return opt_zone_; } inline common::ObTabletID &get_tablet_id() { return opt_tablet_id_; } + inline int64_t &get_ls_id() { return opt_ls_id_; } inline int push_server(const common::ObAddr& server) { return opt_server_list_.push_back(server); } TO_STRING_KV(N_STMT_TYPE, ((int)stmt_type_), K_(major_freeze), - K(opt_server_list_), K(opt_tenant_ids_), K(opt_tablet_id_)); + K(opt_server_list_), K(opt_tenant_ids_), K(opt_tablet_id_), K(opt_ls_id_)); private: bool major_freeze_; // for major_freeze, it is ignore server list @@ -68,8 +69,10 @@ private: common::ObSArray opt_tenant_ids_; // for minor_freeze only common::ObZone opt_zone_; + // for minor_freeze only common::ObTabletID opt_tablet_id_; + int64_t opt_ls_id_; }; class ObFlushCacheStmt : public ObSystemCmdStmt diff --git a/src/sql/session/ob_sql_session_mgr.cpp b/src/sql/session/ob_sql_session_mgr.cpp index a33d6c71b..6703ce15b 100644 --- a/src/sql/session/ob_sql_session_mgr.cpp +++ b/src/sql/session/ob_sql_session_mgr.cpp @@ -28,6 +28,7 @@ #include "sql/monitor/ob_security_audit_utils.h" #include "sql/session/ob_user_resource_mgr.h" #include "sql/monitor/full_link_trace/ob_flt_control_info_mgr.h" +#include "storage/concurrency_control/ob_multi_version_garbage_collector.h" using namespace oceanbase::common; using namespace oceanbase::sql; @@ -470,6 +471,21 @@ void ObSQLSessionMgr::try_check_session() } } +int ObSQLSessionMgr::get_min_active_snapshot_version(share::SCN &snapshot_version) +{ + int ret = OB_SUCCESS; + + concurrency_control::GetMinActiveSnapshotVersionFunctor min_active_txn_version_getter; + + if (OB_FAIL(for_each_session(min_active_txn_version_getter))) { + LOG_WARN("fail to get min active snapshot version", K(ret)); + } else { + snapshot_version = min_active_txn_version_getter.get_min_active_snapshot_version(); + } + + return ret; +} + int ObSQLSessionMgr::check_session_leak() { int ret = OB_SUCCESS; diff --git a/src/sql/session/ob_sql_session_mgr.h b/src/sql/session/ob_sql_session_mgr.h index 5741be38e..072070f3d 100644 --- a/src/sql/session/ob_sql_session_mgr.h +++ b/src/sql/session/ob_sql_session_mgr.h @@ -125,6 +125,9 @@ public: virtual void runTimerTask(); void try_check_session(); + // get min active snapshot version for all session + int get_min_active_snapshot_version(share::SCN &snapshot_version); + //used for guarantee the unique sessid when observer generates sessid static uint64_t extract_server_id(uint32_t sessid); static bool is_server_sessid(uint32_t sessid) {return SERVER_SESSID_TAG & sessid;} diff --git a/src/storage/CMakeLists.txt b/src/storage/CMakeLists.txt index 8c6afbbb3..507fc79b7 100644 --- a/src/storage/CMakeLists.txt +++ b/src/storage/CMakeLists.txt @@ -298,6 +298,7 @@ ob_set_subtarget(ob_storage tx_table tx_table/ob_tx_ctx_memtable.cpp tx_table/ob_tx_ctx_memtable_mgr.cpp tx_table/ob_tx_ctx_table.cpp + tx_table/ob_tx_data_hash_map.cpp tx_table/ob_tx_data_memtable.cpp tx_table/ob_tx_data_memtable_mgr.cpp tx_table/ob_tx_data_table.cpp @@ -452,6 +453,10 @@ ob_set_subtarget(ob_storage compaction compaction/ob_partition_rows_merger.cpp ) +ob_set_subtarget(ob_storage concurrency_control + concurrency_control/ob_multi_version_garbage_collector.cpp +) + ob_set_subtarget(ob_storage memtable memtable/ob_lock_wait_mgr.cpp memtable/ob_memtable.cpp @@ -463,6 +468,7 @@ ob_set_subtarget(ob_storage memtable memtable/ob_multi_source_data.cpp memtable/ob_redo_log_generator.cpp memtable/ob_row_compactor.cpp + memtable/ob_concurrent_control.cpp ) ob_set_subtarget(ob_storage memtable_mvcc diff --git a/src/storage/access/ob_dml_param.cpp b/src/storage/access/ob_dml_param.cpp index 788c24a12..526a2a659 100644 --- a/src/storage/access/ob_dml_param.cpp +++ b/src/storage/access/ob_dml_param.cpp @@ -246,7 +246,8 @@ DEF_TO_STRING(ObDMLBaseParam) K_(is_ignore), K_(prelock), KPC_(encrypt_meta), - K_(is_batch_stmt)); + K_(is_batch_stmt), + K_(write_flag)); J_OBJ_END(); return pos; } diff --git a/src/storage/access/ob_dml_param.h b/src/storage/access/ob_dml_param.h index 505ebf055..3ce744b4f 100644 --- a/src/storage/access/ob_dml_param.h +++ b/src/storage/access/ob_dml_param.h @@ -181,7 +181,8 @@ struct ObDMLBaseParam encrypt_meta_(NULL), encrypt_meta_legacy_(), spec_seq_no_(-1), - snapshot_() + snapshot_(), + write_flag_() { } @@ -209,6 +210,8 @@ struct ObDMLBaseParam int64_t spec_seq_no_; // transaction snapshot transaction::ObTxReadSnapshot snapshot_; + // write flag for inner write processing + concurrent_control::ObWriteFlag write_flag_; bool is_valid() const { return (timeout_ > 0 && schema_version_ >= 0); } DECLARE_TO_STRING; }; diff --git a/src/storage/blocksstable/encoding/ob_micro_block_decoder.cpp b/src/storage/blocksstable/encoding/ob_micro_block_decoder.cpp index 2955e66f5..82498405c 100644 --- a/src/storage/blocksstable/encoding/ob_micro_block_decoder.cpp +++ b/src/storage/blocksstable/encoding/ob_micro_block_decoder.cpp @@ -1814,12 +1814,11 @@ int ObMicroBlockDecoder::get_row_count(int64_t &row_count) int ObMicroBlockDecoder::get_multi_version_info( const int64_t row_idx, const int64_t schema_rowkey_cnt, - ObMultiVersionRowFlag &flag, - transaction::ObTransID &trans_id, + const ObRowHeader *&row_header, int64_t &version, int64_t &sql_sequence) { - UNUSEDx(row_idx, schema_rowkey_cnt, flag, trans_id, version, sql_sequence); + UNUSEDx(row_idx, schema_rowkey_cnt, row_header, version, sql_sequence); return OB_NOT_SUPPORTED; } diff --git a/src/storage/blocksstable/encoding/ob_micro_block_decoder.h b/src/storage/blocksstable/encoding/ob_micro_block_decoder.h index 6e766c2b4..e7fd82a77 100644 --- a/src/storage/blocksstable/encoding/ob_micro_block_decoder.h +++ b/src/storage/blocksstable/encoding/ob_micro_block_decoder.h @@ -203,8 +203,7 @@ public: virtual int get_multi_version_info( const int64_t row_idx, const int64_t schema_rowkey_cnt, - ObMultiVersionRowFlag &flag, - transaction::ObTransID &trans_id, + const ObRowHeader *&row_header, int64_t &version, int64_t &sql_sequence); int compare_rowkey( diff --git a/src/storage/blocksstable/ob_imicro_block_reader.h b/src/storage/blocksstable/ob_imicro_block_reader.h index 508abc8dd..2977603b8 100644 --- a/src/storage/blocksstable/ob_imicro_block_reader.h +++ b/src/storage/blocksstable/ob_imicro_block_reader.h @@ -273,9 +273,8 @@ public: virtual int get_multi_version_info( const int64_t row_idx, const int64_t schema_rowkey_cnt, - ObMultiVersionRowFlag &flag, - transaction::ObTransID &trans_id, - int64_t &version, + const ObRowHeader *&row_header, + int64_t &trans_version, int64_t &sql_sequence) = 0; int locate_range( const ObDatumRange &range, diff --git a/src/storage/blocksstable/ob_micro_block_reader.cpp b/src/storage/blocksstable/ob_micro_block_reader.cpp index e4e4c69c7..576e74183 100644 --- a/src/storage/blocksstable/ob_micro_block_reader.cpp +++ b/src/storage/blocksstable/ob_micro_block_reader.cpp @@ -467,13 +467,12 @@ int ObMicroBlockReader::get_row_count(int64_t &row_count) int ObMicroBlockReader::get_multi_version_info( const int64_t row_idx, const int64_t schema_rowkey_cnt, - ObMultiVersionRowFlag &flag, - transaction::ObTransID &trans_id, + const ObRowHeader *&row_header, int64_t &trans_version, int64_t &sql_sequence) { int ret = OB_SUCCESS; - const ObRowHeader *row_header = nullptr; + row_header = nullptr; if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret)); @@ -491,11 +490,10 @@ int ObMicroBlockReader::get_multi_version_info( LOG_WARN("fail to setup row", K(ret), K(row_idx), K(index_data_[row_idx + 1]), K(index_data_[row_idx]), KP(data_begin_)); } else { - trans_id = row_header->get_trans_id(); - flag = row_header->get_row_multi_version_flag(); - ObStorageDatum datum; - const int64_t read_col_idx = flag.is_uncommitted_row() ? schema_rowkey_cnt + 1 : schema_rowkey_cnt; + const int64_t read_col_idx = + row_header->get_row_multi_version_flag().is_uncommitted_row() + ? schema_rowkey_cnt + 1 : schema_rowkey_cnt; if (OB_FAIL(flat_row_reader_.read_column( data_begin_ + index_data_[row_idx], index_data_[row_idx + 1] - index_data_[row_idx], @@ -503,9 +501,10 @@ int ObMicroBlockReader::get_multi_version_info( datum))) { LOG_WARN("fail to read column", K(ret), K(read_col_idx)); } else { - if (!flag.is_uncommitted_row()) { // get trans_version for committed row + if (!row_header->get_row_multi_version_flag().is_uncommitted_row()) { + // get trans_version for committed row sql_sequence = 0; - trans_version = flag.is_ghost_row() ? 0 : -datum.get_int(); + trans_version = row_header->get_row_multi_version_flag().is_ghost_row() ? 0 : -datum.get_int(); } else { // get sql_sequence for uncommitted row trans_version = INT64_MAX; diff --git a/src/storage/blocksstable/ob_micro_block_reader.h b/src/storage/blocksstable/ob_micro_block_reader.h index 4ad517daf..11fc059a2 100644 --- a/src/storage/blocksstable/ob_micro_block_reader.h +++ b/src/storage/blocksstable/ob_micro_block_reader.h @@ -76,8 +76,7 @@ public: int get_multi_version_info( const int64_t row_idx, const int64_t schema_rowkey_cnt, - ObMultiVersionRowFlag &flag, - transaction::ObTransID &trans_id, + const ObRowHeader *&row_header, int64_t &version, int64_t &sql_sequence); // Filter interface for filter pushdown diff --git a/src/storage/blocksstable/ob_micro_block_row_lock_checker.cpp b/src/storage/blocksstable/ob_micro_block_row_lock_checker.cpp index 7659cdb84..1a2f9969f 100644 --- a/src/storage/blocksstable/ob_micro_block_row_lock_checker.cpp +++ b/src/storage/blocksstable/ob_micro_block_row_lock_checker.cpp @@ -37,9 +37,8 @@ int ObMicroBlockRowLockChecker::get_next_row(const ObDatumRow *&row) ret = OB_ERR_UNEXPECTED; LOG_WARN("Unexpected null param", K(ret), KP_(read_info), KP_(lock_state)); } else { - transaction::ObTransID trans_id; + const ObRowHeader *row_header = nullptr; int64_t sql_sequence = 0; - ObMultiVersionRowFlag flag; const int64_t rowkey_cnt = read_info_->get_schema_rowkey_count(); memtable::ObMvccAccessCtx &ctx = context_->store_ctx_->mvcc_acc_ctx_; const transaction::ObTransID &read_trans_id = ctx.get_tx_id(); @@ -53,15 +52,18 @@ int ObMicroBlockRowLockChecker::get_next_row(const ObDatumRow *&row) } else if (OB_FAIL(reader_->get_multi_version_info( current_, rowkey_cnt, - flag, - trans_id, + row_header, trans_version, sql_sequence))) { - LOG_WARN("failed to get multi version info", K(ret), K_(current), K(flag), K(trans_id), + LOG_WARN("failed to get multi version info", K(ret), K_(current), KPC(row_header), KPC_(lock_state), K(sql_sequence), K_(macro_id)); + } else if (OB_ISNULL(row_header)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("row header is null", K(ret)); + // TODO(handora.qc): fix it } else if (OB_FAIL(lock_state_->trans_version_.convert_for_tx(trans_version))) { LOG_ERROR("convert failed", K(ret), K(trans_version)); - } else if (flag.is_uncommitted_row()) { + } else if (row_header->get_row_multi_version_flag().is_uncommitted_row()) { ObTxTableGuard tx_table_guard = ctx.get_tx_table_guard(); ObTxTable *tx_table = nullptr; int64 read_epoch = ObTxTable::INVALID_READ_EPOCH; @@ -70,21 +72,26 @@ int ObMicroBlockRowLockChecker::get_next_row(const ObDatumRow *&row) LOG_ERROR("tx table guard is invalid", KR(ret), K(ctx)); } else if (FALSE_IT(tx_table = tx_table_guard.get_tx_table())) { } else if (FALSE_IT(read_epoch = tx_table_guard.epoch())) { - } else if (OB_FAIL( - tx_table->check_row_locked(read_trans_id, trans_id, sql_sequence, read_epoch, *lock_state_))) { + } else if (OB_FAIL(tx_table->check_row_locked(read_trans_id, + row_header->get_trans_id(), + sql_sequence, + read_epoch, + *lock_state_))) { + } else if (lock_state_->is_locked_) { + lock_state_->lock_dml_flag_ = row_header->get_row_flag().get_dml_flag(); } - STORAGE_LOG(DEBUG, "check row lock", K(ret), KPC_(range), K(read_trans_id), K(trans_id), + STORAGE_LOG(DEBUG, "check row lock", K(ret), KPC_(range), K(read_trans_id), KPC(row_header), K(sql_sequence), KPC_(lock_state)); if (SCN::min_scn() != lock_state_->trans_version_ || // trans is commit lock_state_->is_locked_) { break; } } else { // committed row - if (flag.is_ghost_row()) { - if (OB_UNLIKELY(!flag.is_last_multi_version_row())) { + if (row_header->get_row_multi_version_flag().is_ghost_row()) { + if (OB_UNLIKELY(!row_header->get_row_multi_version_flag().is_last_multi_version_row())) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("Unexpected row flag", K(ret), K(flag)); + LOG_WARN("Unexpected row flag", K(ret), KPC(row_header)); } else { ret = OB_ITER_END; } diff --git a/src/storage/blocksstable/ob_micro_block_row_scanner.cpp b/src/storage/blocksstable/ob_micro_block_row_scanner.cpp index 4001ae0cf..8260fba39 100644 --- a/src/storage/blocksstable/ob_micro_block_row_scanner.cpp +++ b/src/storage/blocksstable/ob_micro_block_row_scanner.cpp @@ -1018,7 +1018,7 @@ int ObMultiVersionMicroBlockRowScanner::inner_inner_get_next_row( } else { ObMultiVersionRowFlag flag; int64_t trans_version = 0; - transaction::ObTransID trans_id; + const ObRowHeader *row_header = nullptr; int64_t sql_sequence = 0; bool can_read = true; bool is_determined_state = false; @@ -1030,17 +1030,20 @@ int ObMultiVersionMicroBlockRowScanner::inner_inner_get_next_row( } else if (OB_FAIL(reader_->get_multi_version_info( current_, read_info_->get_schema_rowkey_count(), - flag, - trans_id, + row_header, trans_version, sql_sequence))) { LOG_WARN("fail to get multi version info", K(ret), K(current_), KPC_(read_info), K(sql_sequence_col_idx_), K_(macro_id)); + } else if (OB_ISNULL(row_header)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("row header is null", K(ret)); + } else if (FALSE_IT(flag = row_header->get_row_multi_version_flag())) { } else if (flag.is_uncommitted_row()) { have_uncommited_row = true; // TODO @lvling check transaction status instead auto &acc_ctx = context_->store_ctx_->mvcc_acc_ctx_; transaction::ObLockForReadArg lock_for_read_arg(acc_ctx, - trans_id, + row_header->get_trans_id(), sql_sequence, context_->query_flag_.read_latest_); @@ -1048,13 +1051,13 @@ int ObMultiVersionMicroBlockRowScanner::inner_inner_get_next_row( can_read, trans_version, is_determined_state))) { - STORAGE_LOG(WARN, "fail to check transaction status", K(ret), K(trans_id), K_(macro_id)); + STORAGE_LOG(WARN, "fail to check transaction status", K(ret), KPC(row_header), K_(macro_id)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(ObGhostRowUtil::is_ghost_row(flag, is_ghost_row_flag))) { - LOG_WARN("fail to check ghost row", K(ret), K_(current), K(trans_id), + LOG_WARN("fail to check ghost row", K(ret), K_(current), KPC(row_header), K(trans_version), K(sql_sequence), K_(macro_id)); } else if (OB_UNLIKELY(is_ghost_row_flag)) { can_read = false; @@ -1677,6 +1680,7 @@ int ObMultiVersionMicroBlockMinorMergeRowScanner::read_uncommitted_row( bool &can_read, const ObDatumRow *&row) { int ret = OB_SUCCESS; + const ObRowHeader *row_header = nullptr; ObMultiVersionRowFlag flag; int64_t trans_version = 0; int64_t sql_sequence = 0; @@ -1685,12 +1689,16 @@ int ObMultiVersionMicroBlockMinorMergeRowScanner::read_uncommitted_row( if (OB_FAIL(reader_->get_multi_version_info( current_, read_info_->get_schema_rowkey_count(), - flag, - read_trans_id_, + row_header, trans_version, sql_sequence))) { LOG_WARN("fail to get multi version info", K(ret), K(current_), KPC_(read_info), K_(macro_id)); + } else if (OB_ISNULL(row_header)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("row header is null", K(ret)); + } else if (FALSE_IT(flag = row_header->get_row_multi_version_flag())) { + } else if (FALSE_IT(read_trans_id_ = row_header->get_trans_id())) { } else if (flag.is_uncommitted_row()) { // uncommitted row bool read_row_flag = false; if (OB_UNLIKELY(read_trans_id_ != last_trans_id_)) { // another uncommitted trans @@ -1959,18 +1967,23 @@ int ObMultiVersionMicroBlockMinorMergeRowScanner::find_uncommitted_row() LOG_DEBUG("find uncommitted row failed", K(ret)); } } else { + const ObRowHeader *row_header = nullptr; ObMultiVersionRowFlag flag; int64_t trans_version = 0; int64_t sql_sequence = 0; if (OB_FAIL(reader_->get_multi_version_info( current_, read_info_->get_schema_rowkey_count(), - flag, - last_trans_id_, // record the trans_id + row_header, trans_version, sql_sequence))) { LOG_WARN("fail to get multi version info", K(ret), K(current_), KPC_(read_info), K_(macro_id)); + } else if (OB_ISNULL(row_header)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("row header is null", K(ret)); + } else if (FALSE_IT(flag = row_header->get_row_multi_version_flag())) { + } else if (FALSE_IT(last_trans_id_ = row_header->get_trans_id())) { } else if (flag.is_uncommitted_row()) { // uncommitted //get trans status & committed_trans_version_ int64_t state; diff --git a/src/storage/blocksstable/ob_sstable_printer.cpp b/src/storage/blocksstable/ob_sstable_printer.cpp index e3d2ca264..31ace3f05 100644 --- a/src/storage/blocksstable/ob_sstable_printer.cpp +++ b/src/storage/blocksstable/ob_sstable_printer.cpp @@ -472,14 +472,14 @@ void ObSSTablePrinter::print_store_row( } } else { // pre-process data for upper trans version calculation - void *p = op_alloc(ObCommitSCNsArray); - ObCommitSCNsArray *commit_versions = new (p) ObCommitSCNsArray(); + void *p = op_alloc(ObCommitVersionsArray); + ObCommitVersionsArray *commit_versions = new (p) ObCommitVersionsArray(); if (OB_FAIL(commit_versions->deserialize(str.ptr(), str.length(), pos))) { STORAGE_LOG(WARN, "deserialize commit versions failed", KR(ret), K(str)); hex_dump(str.ptr(), str.length(), true, OB_LOG_LEVEL_WARN); } else { - ObCommitSCNsArray::print_to_stderr(*commit_versions); + ObCommitVersionsArray::print_to_stderr(*commit_versions); } } } else { diff --git a/src/storage/compaction/ob_partition_merge_policy.cpp b/src/storage/compaction/ob_partition_merge_policy.cpp index 2a5307ee5..615f69172 100644 --- a/src/storage/compaction/ob_partition_merge_policy.cpp +++ b/src/storage/compaction/ob_partition_merge_policy.cpp @@ -187,6 +187,7 @@ int ObPartitionMergePolicy::get_mini_merge_tables( && OB_FAIL(deal_with_minor_result(merge_type, ls, tablet, result))) { LOG_WARN("failed to deal with minor merge result", K(ret)); } + return ret; } @@ -281,6 +282,7 @@ int ObPartitionMergePolicy::find_mini_merge_tables( result.schedule_major_ = true; } } + return ret; } diff --git a/src/storage/compaction/ob_partition_parallel_merge_ctx.h b/src/storage/compaction/ob_partition_parallel_merge_ctx.h index 7e356ec50..3a97b536c 100644 --- a/src/storage/compaction/ob_partition_parallel_merge_ctx.h +++ b/src/storage/compaction/ob_partition_parallel_merge_ctx.h @@ -50,7 +50,7 @@ public: virtual ~ObParallelMergeCtx(); void reset(); bool is_valid() const; - int init(compaction::ObTabletMergeCtx &merge_ctx); + OB_NOINLINE int init(compaction::ObTabletMergeCtx &merge_ctx);// will be mocked in mittest int init(const compaction::ObMediumCompactionInfo &medium_info); OB_INLINE int64_t get_concurrent_cnt() const { return concurrent_cnt_; } int get_merge_range(const int64_t parallel_idx, blocksstable::ObDatumRange &merge_range); @@ -65,7 +65,7 @@ private: static const int64_t PARALLEL_MERGE_TARGET_TASK_CNT = 20; //TODO @hanhui parallel in ai int init_serial_merge(); - int init_parallel_mini_merge(compaction::ObTabletMergeCtx &merge_ctx); + OB_NOINLINE int init_parallel_mini_merge(compaction::ObTabletMergeCtx &merge_ctx);// will be mocked in mittest int init_parallel_mini_minor_merge(compaction::ObTabletMergeCtx &merge_ctx); int init_parallel_major_merge(compaction::ObTabletMergeCtx &merge_ctx); int calc_mini_minor_parallel_degree(const int64_t tablet_size, diff --git a/src/storage/compaction/ob_tablet_merge_ctx.cpp b/src/storage/compaction/ob_tablet_merge_ctx.cpp index b51986745..ea764c22c 100644 --- a/src/storage/compaction/ob_tablet_merge_ctx.cpp +++ b/src/storage/compaction/ob_tablet_merge_ctx.cpp @@ -394,6 +394,7 @@ const char *ObCompactionTimeGuard::ObTabletCompactionEventStr[] = { "COMPACTION_POLICY", "GET_SCHEMA", "CALC_PROGRESSIVE_PARAM", + "PRE_PROCESS_TX_TABLE", "GET_PARALLEL_RANGE", "EXECUTE", "CREATE_SSTABLE", diff --git a/src/storage/compaction/ob_tablet_merge_ctx.h b/src/storage/compaction/ob_tablet_merge_ctx.h index 81e3adcee..b1175a285 100644 --- a/src/storage/compaction/ob_tablet_merge_ctx.h +++ b/src/storage/compaction/ob_tablet_merge_ctx.h @@ -116,6 +116,7 @@ public: COMPACTION_POLICY, GET_TABLE_SCHEMA, CALC_PROGRESSIVE_PARAM, + PRE_PROCESS_TX_TABLE, GET_PARALLEL_RANGE, EXECUTE, CREATE_SSTABLE, diff --git a/src/storage/compaction/ob_tenant_freeze_info_mgr.cpp b/src/storage/compaction/ob_tenant_freeze_info_mgr.cpp index e8a64571b..ff670641e 100644 --- a/src/storage/compaction/ob_tenant_freeze_info_mgr.cpp +++ b/src/storage/compaction/ob_tenant_freeze_info_mgr.cpp @@ -34,6 +34,7 @@ #include "share/ob_zone_merge_table_operator.h" #include "storage/compaction/ob_server_compaction_event_history.h" #include "storage/compaction/ob_tenant_tablet_scheduler.h" +#include "storage/concurrency_control/ob_multi_version_garbage_collector.h" #include "storage/tx_storage/ob_ls_map.h" #include "storage/tx_storage/ob_ls_service.h" @@ -420,6 +421,39 @@ int ObTenantFreezeInfoMgr::get_multi_version_duration(int64_t &duration) const return ret; } +int64_t ObTenantFreezeInfoMgr::get_min_reserved_snapshot_for_tx() +{ + int ret = OB_SUCCESS; + int64_t snapshot_version = INT64_MAX; + uint64_t data_version = 0; + + // is_gc_disabled means whether gc using globally reserved snapshot is disabled, + // and it may be because of disk usage or lost connection to inner table + bool is_gc_disabled = MTL(concurrency_control::ObMultiVersionGarbageCollector *)-> + is_gc_disabled(); + + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + + if (OB_FAIL(GET_MIN_DATA_VERSION(gen_meta_tenant_id(MTL_ID()), + data_version))) { + STORAGE_LOG(WARN, "get min data version failed", KR(ret), + K(gen_meta_tenant_id(MTL_ID()))); + // we disable the gc when fetch min data version failed + is_gc_disabled = true; + } + + if (data_version >= DATA_VERSION_4_1_0_0 + && tenant_config->_mvcc_gc_using_min_txn_snapshot + && !is_gc_disabled) { + share::SCN snapshot_for_active_tx = + MTL(concurrency_control::ObMultiVersionGarbageCollector *)-> + get_reserved_snapshot_for_active_txn(); + snapshot_version = snapshot_for_active_tx.get_val_for_tx(); + } + + return snapshot_version; +} + int ObTenantFreezeInfoMgr::get_min_reserved_snapshot( const ObTabletID &tablet_id, const int64_t merged_version, @@ -450,8 +484,11 @@ int ObTenantFreezeInfoMgr::get_min_reserved_snapshot( ret = OB_SUCCESS; } } + snapshot_version = std::max(0L, snapshot_gc_ts_ - duration * 1000L * 1000L * 1000L); + snapshot_version = std::min(snapshot_version, get_min_reserved_snapshot_for_tx()); snapshot_version = std::min(snapshot_version, freeze_info.freeze_version); + for (int64_t i = 0; i < snapshots.count() && OB_SUCC(ret); ++i) { bool related = false; const ObSnapshotInfo &snapshot = snapshots.at(i); @@ -497,12 +534,21 @@ int ObTenantFreezeInfoMgr::diagnose_min_reserved_snapshot( ret = OB_SUCCESS; } } + snapshot_version = std::max(0L, snapshot_gc_ts_ - duration * 1000L * 1000L * 1000L); snapshot_from_type = "undo_retention"; + + const int64_t snapshot_version_for_tx = get_min_reserved_snapshot_for_tx(); + if (snapshot_version_for_tx < snapshot_version) { + snapshot_version = snapshot_version_for_tx; + snapshot_from_type = "snapshot_for_tx"; + } + if (freeze_info.freeze_version < snapshot_version) { snapshot_version = freeze_info.freeze_version; snapshot_from_type = "major_freeze_ts"; } + for (int64_t i = 0; i < snapshots.count() && OB_SUCC(ret); ++i) { bool related = false; const ObSnapshotInfo &snapshot = snapshots.at(i); diff --git a/src/storage/compaction/ob_tenant_freeze_info_mgr.h b/src/storage/compaction/ob_tenant_freeze_info_mgr.h index a7849c815..3cbb2d196 100644 --- a/src/storage/compaction/ob_tenant_freeze_info_mgr.h +++ b/src/storage/compaction/ob_tenant_freeze_info_mgr.h @@ -94,6 +94,7 @@ public: int get_neighbour_major_freeze(const int64_t snapshot_version, NeighbourFreezeInfo &info); + int64_t get_min_reserved_snapshot_for_tx(); int get_min_reserved_snapshot( const ObTabletID &tablet_id, const int64_t merged_version, diff --git a/src/storage/compaction/ob_tenant_tablet_scheduler.cpp b/src/storage/compaction/ob_tenant_tablet_scheduler.cpp old mode 100644 new mode 100755 diff --git a/src/storage/compaction/ob_tx_table_merge_task.cpp b/src/storage/compaction/ob_tx_table_merge_task.cpp old mode 100644 new mode 100755 index 664aca9eb..e0dcdb395 --- a/src/storage/compaction/ob_tx_table_merge_task.cpp +++ b/src/storage/compaction/ob_tx_table_merge_task.cpp @@ -104,6 +104,23 @@ int ObTxTableMergePrepareTask::init() return ret; } +int ObTxTableMergePrepareTask::pre_process_tx_data_table_merge_(ObTabletMergeCtx &ctx) +{ + int ret = OB_SUCCESS; + + if (is_mini_merge(ctx.param_.merge_type_)) { + common::ObIArray &tables = ctx.tables_handle_.get_tables(); + for (int i = 0; OB_SUCC(ret) && i < tables.count(); i++) { + if (OB_FAIL(static_cast(tables.at(i))->pre_process_for_merge())) { + LOG_WARN("do pre process for tx data table merge failed.", K(ret), K(ctx.param_), + KPC(tables.at(i))); + } + } + } + ctx.time_guard_.click(ObCompactionTimeGuard::PRE_PROCESS_TX_TABLE); + return ret; +} + int ObTxTableMergePrepareTask::inner_init_ctx(ObTabletMergeCtx &ctx, bool &skip_merge_task_flag) { int ret = OB_SUCCESS; @@ -134,6 +151,9 @@ int ObTxTableMergePrepareTask::inner_init_ctx(ObTabletMergeCtx &ctx, bool &skip_ LOG_WARN("failed to set basic info to ctx", K(ret), K(get_merge_table_result), K(ctx)); } else if (OB_FAIL(ctx.get_storage_schema_to_merge(get_merge_table_result.handle_, false/*get_schema_on_memtable*/))) { LOG_WARN("failed to get storage schema", K(ret), K(get_merge_table_result), K(ctx)); + } else if (LS_TX_DATA_TABLET == ctx.param_.tablet_id_ + && OB_FAIL(pre_process_tx_data_table_merge_(ctx))) { + LOG_WARN("pre process tx data table for merge failed.", KR(ret), K(ctx.param_)); } else { ctx.progressive_merge_num_ = 0; ctx.is_full_merge_ = true; diff --git a/src/storage/compaction/ob_tx_table_merge_task.h b/src/storage/compaction/ob_tx_table_merge_task.h old mode 100644 new mode 100755 index a02a2689a..08334569a --- a/src/storage/compaction/ob_tx_table_merge_task.h +++ b/src/storage/compaction/ob_tx_table_merge_task.h @@ -36,6 +36,7 @@ public: virtual ~ObTxTableMergePrepareTask(); int init(); private: + int pre_process_tx_data_table_merge_(ObTabletMergeCtx &ctx); virtual int inner_init_ctx(ObTabletMergeCtx &ctx, bool &skip_merge_task_flag) override; private: DISALLOW_COPY_AND_ASSIGN(ObTxTableMergePrepareTask); diff --git a/src/storage/concurrency_control/ob_multi_version_garbage_collector.cpp b/src/storage/concurrency_control/ob_multi_version_garbage_collector.cpp new file mode 100644 index 000000000..a0e20f365 --- /dev/null +++ b/src/storage/concurrency_control/ob_multi_version_garbage_collector.cpp @@ -0,0 +1,1328 @@ +/** + * 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/concurrency_control/ob_multi_version_garbage_collector.h" +#include "storage/tx_storage/ob_ls_service.h" +#include "storage/tx/ob_trans_service.h" +#include "src/storage/tx/ob_ts_mgr.h" +#include "storage/tx/wrs/ob_weak_read_util.h" + +namespace oceanbase +{ +namespace concurrency_control +{ + +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_PRECISION = 1_s; +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_RETRY_INTERVAL = 1_min; +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_EXEC_INTERVAL = 10 * GARBAGE_COLLECT_RETRY_INTERVAL; +int64_t ObMultiVersionGarbageCollector::GARBAGE_COLLECT_RECLAIM_DURATION = 3 * GARBAGE_COLLECT_EXEC_INTERVAL; + +ObMultiVersionGarbageCollector::ObMultiVersionGarbageCollector() + : timer_(), + timer_handle_(), + last_study_timestamp_(0), + last_refresh_timestamp_(0), + last_reclaim_timestamp_(0), + has_error_when_study_(false), + refresh_error_too_long_(false), + has_error_when_reclaim_(false), + gc_is_disabled_(false), + global_reserved_snapshot_(share::SCN::min_scn()), + is_inited_(false) {} + +ObMultiVersionGarbageCollector::~ObMultiVersionGarbageCollector() {} + +int ObMultiVersionGarbageCollector::mtl_init(ObMultiVersionGarbageCollector *&m) +{ + return m->init(); +} + +int ObMultiVersionGarbageCollector::init() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + MVCC_LOG(WARN, "ObMultiVersionGarbageCollector init twice", K(ret), KP(this)); + } else { + last_study_timestamp_ = 0; + last_refresh_timestamp_ = 0; + last_reclaim_timestamp_ = 0; + has_error_when_study_ = false; + refresh_error_too_long_ = false; + has_error_when_reclaim_ = false; + gc_is_disabled_ = false; + global_reserved_snapshot_ = share::SCN::min_scn(); + is_inited_ = true; + MVCC_LOG(INFO, "multi version garbage collector init", KP(this)); + } + return ret; +} + +void ObMultiVersionGarbageCollector::cure() +{ + last_study_timestamp_ = 0; + last_refresh_timestamp_ = 0; + last_reclaim_timestamp_ = 0; + has_error_when_study_ = false; + refresh_error_too_long_ = false; + has_error_when_reclaim_ = false; + gc_is_disabled_ = false; + global_reserved_snapshot_ = share::SCN::min_scn(); +} + +int ObMultiVersionGarbageCollector::start() +{ + int ret = OB_SUCCESS; + + if(!is_inited_) { + ret = OB_NOT_INIT; + MVCC_LOG(ERROR, "has not been inited", KR(ret), K(MTL_ID())); + } else if (OB_FAIL(timer_.init_and_start(1 /*worker_num*/, + GARBAGE_COLLECT_PRECISION /*precision*/, + "MultiVersionGC" /*label*/))) { + MVCC_LOG(ERROR, "fail to init and start timer", KR(ret), KPC(this)); + } else if (OB_FAIL(timer_.schedule_task_repeat( + timer_handle_, + GARBAGE_COLLECT_RETRY_INTERVAL, /*interval*/ + [this]() { /*task*/ + int ret = OB_SUCCESS; + uint64_t data_version = 0; + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + if (!tenant_config->_mvcc_gc_using_min_txn_snapshot) { + cure(); + } else if (OB_FAIL(GET_MIN_DATA_VERSION(gen_meta_tenant_id(MTL_ID()), + data_version))) { + MVCC_LOG(WARN, "get min data version failed", KR(ret), + K(gen_meta_tenant_id(MTL_ID()))); + } else if (data_version >= DATA_VERSION_4_1_0_0) { + // compatibility is important + (void)repeat_study(); + (void)repeat_refresh(); + (void)repeat_reclaim(); + } + return false; }))) { + MVCC_LOG(ERROR, "schedule repeat task failed", KR(ret), KPC(this)); + } else { + MVCC_LOG(INFO, "multi version garbage collector start", KPC(this), + K(GARBAGE_COLLECT_RETRY_INTERVAL), K(GARBAGE_COLLECT_EXEC_INTERVAL), + K(GARBAGE_COLLECT_PRECISION), K(GARBAGE_COLLECT_RECLAIM_DURATION)); + } + + return ret; +} + +int ObMultiVersionGarbageCollector::stop() +{ + int ret = OB_SUCCESS; + + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + MVCC_LOG(WARN, "ObCheckPointService is not initialized", K(ret)); + } else { + ObTimeGuard timeguard(__func__, 1 * 1000 * 1000); + (void)timer_handle_.stop_and_wait(); + (void)timer_.stop_and_wait(); + last_study_timestamp_ = 0; + last_refresh_timestamp_ = 0; + last_reclaim_timestamp_ = 0; + has_error_when_study_ = false; + refresh_error_too_long_ = false; + has_error_when_reclaim_ = false; + gc_is_disabled_ = false; + global_reserved_snapshot_ = share::SCN::min_scn(); + is_inited_ = false; + MVCC_LOG(INFO, "multi version garbage collector stop", KPC(this)); + } + + return ret; +} + +void ObMultiVersionGarbageCollector::wait() +{ + MVCC_LOG(INFO, "multi version garbage collector wait", KPC(this)); +} + +void ObMultiVersionGarbageCollector::destroy() +{ + MVCC_LOG(INFO, "multi version garbage collector destroy", KPC(this)); +} + +// study means learning for the different ObMultiVersionSnapshotType and +// reporting to the inner table. And the repeat_study will study each time when +// meeting the error or meeting the time requirement. +void ObMultiVersionGarbageCollector::repeat_study() +{ + int ret = OB_SUCCESS; + const int64_t current_timestamp = ObClockGenerator::getRealClock(); + + if (has_error_when_study_ // enconter error during last study + // study every 10 min(default of GARBAGE_COLLECT_EXEC_INTERVAL) + || current_timestamp - last_study_timestamp_ > GARBAGE_COLLECT_EXEC_INTERVAL) { + if (OB_FAIL(study())) { + has_error_when_study_ = true; + if (current_timestamp - last_study_timestamp_ > 10 * GARBAGE_COLLECT_EXEC_INTERVAL + && 0 != last_study_timestamp_ + // for mock or test that change GARBAGE_COLLECT_EXEC_INTERVAL to a small value + && current_timestamp - last_study_timestamp_ > 10 * 10_min) { + MVCC_LOG(ERROR, "repeat study failed too much time", K(ret), + KPC(this), K(current_timestamp)); + } else { + MVCC_LOG(WARN, "repeat study failed, we will retry immediately", K(ret), + KPC(this), K(current_timestamp)); + } + } else { + has_error_when_study_ = false; + last_study_timestamp_ = common::ObTimeUtility::current_time(); + MVCC_LOG(INFO, "repeat study successfully", K(ret), KPC(this), + K(current_timestamp), K(GARBAGE_COLLECT_EXEC_INTERVAL)); + } + } else { + MVCC_LOG(INFO, "skip repeat study", K(ret), KPC(this), + K(current_timestamp), K(GARBAGE_COLLECT_EXEC_INTERVAL)); + } +} + +// collect means collection for the different ObMultiVersionSnapshotType from +// the inner table. And the repeat_collect will study each time when meeting the +// time requirement. +void ObMultiVersionGarbageCollector::repeat_refresh() +{ + int ret = OB_SUCCESS; + const int64_t current_timestamp = ObClockGenerator::getRealClock(); + + // collect every 1 min(default of GARBAGE_COLLECT_RETRY_INTERVAL) + if (OB_FAIL(refresh_())) { + if (is_refresh_fail() || + (current_timestamp - last_refresh_timestamp_ > 30 * GARBAGE_COLLECT_RETRY_INTERVAL + && 0 != last_refresh_timestamp_ + // for mock or test that change GARBAGE_COLLECT_RETRY_INTERVAL to a small value + && current_timestamp - last_refresh_timestamp_ > 30 * 1_min)) { + // the server may cannot contact to the inner table and prevent reserved + // snapshot from advancing. We think the multi-version data on this server + // is not reachable for all active txns, so we gives up to use follow the + // rules of multi-version garbage collector and use the undo_retention and + // snapshot_gc_ts as the new mechanism to guarantee the recycle of data. + refresh_error_too_long_ = true; + MVCC_LOG(ERROR, "repeat refresh failed too much time", K(ret), + KPC(this), K(current_timestamp)); + } else { + MVCC_LOG(WARN, "repeat refresh failed, we will retry immediately", K(ret), + KPC(this), K(current_timestamp)); + } + } else { + refresh_error_too_long_ = false; + last_refresh_timestamp_ = common::ObTimeUtility::current_time(); + MVCC_LOG(INFO, "repeat refresh successfully", K(ret), KPC(this), + K(current_timestamp), K(GARBAGE_COLLECT_RETRY_INTERVAL)); + } +} + +// reclaim means collecting and reclaiming the expired entries in the inner +// table. And the repeat_reclaim will relaim each time we meeting the error or +// meeting the time requirement. +void ObMultiVersionGarbageCollector::repeat_reclaim() +{ + int ret = OB_SUCCESS; + const int64_t current_timestamp = ObClockGenerator::getRealClock(); + + if (has_error_when_reclaim_ // enconter error during last reclaim + // reclaim every 10 min(default of GARBAGE_COLLECT_EXEC_INTERVAL) + || current_timestamp - last_reclaim_timestamp_ > GARBAGE_COLLECT_EXEC_INTERVAL) { + if (OB_FAIL(reclaim())) { + has_error_when_reclaim_ = true; + if (current_timestamp - last_reclaim_timestamp_ > 10 * GARBAGE_COLLECT_EXEC_INTERVAL + && 0 != last_reclaim_timestamp_ + // for mock or test that change GARBAGE_COLLECT_EXEC_INTERVAL to a small value + && current_timestamp - last_reclaim_timestamp_ > 10 * 10_min) { + MVCC_LOG(ERROR, "repeat reclaim failed too much time", K(ret), + KPC(this), K(current_timestamp)); + } else { + MVCC_LOG(WARN, "repeat reclaim failed, we will retry immediately", K(ret), + KPC(this), K(current_timestamp)); + } + } else { + has_error_when_reclaim_ = false; + last_reclaim_timestamp_ = common::ObTimeUtility::current_time(); + MVCC_LOG(INFO, "repeat reclaim successfully", K(ret), KPC(this), + K(current_timestamp), K(GARBAGE_COLLECT_EXEC_INTERVAL)); + } + } else { + MVCC_LOG(INFO, "skip repeat reclaim", K(ret), KPC(this), + K(current_timestamp), K(GARBAGE_COLLECT_EXEC_INTERVAL)); + } +} + +// According to the requirement of the multi-version garbage collector and the +// document https://yuque.antfin-inc.com/ob/transaction/pqhlx4. We need collect +// four timestamp from each OceanBase node: +// 1. The timestamp of the minimum unallocation GTS +// 2. The timestamp of the minimum unallocation WRS +// 3. The maximum commit timestamp of each node +// 4. The minimum snapshot of the active txns of each node +// +// | | +// +--t1----C----------------------t2--------->| Machine1 +// | | +// | | +// +-------------------C---------------------->| Machine2 +// | | +// | | +// +----------------------------------C------->| Machine3 +// | | +// GARBAGE_COLLECT_EXEC_INTERVAL +// +// Let's watch for the above example, three machines may each report(the action +// "C"" in the above picture) its timestamps to inner table at different times +// during the GARBAGE_COLLECT_EXEC_INTERVAL. We can think the questions basing on +// one of the instance. Let's go for the machine1: +// (Let's think that a txn must only starts on one of the machine(called TxDesc)) +// 1. If the txn starts before the report started: +// a. If the txn has finished before C, we need not take it into consideration. +// b. If the txn has not finished, we will consider it with the above timestamp 4. +// 2. If the txn starts after the report started: +// a. The txn using GTS as snapshot, we will consider it with the above timestamp 1. +// b. The txn using WRS as snapshot, we will consider it with the above timestamp 2. +// c. The txn using max committed version as snapshot, we will consider it with the +// above timestamp 3. +// +// So if we generalize all machines using the above rules, all txns started on known +// machine will be taken into condsideration based on our alogorithm +// +// NB: So we must insert or update the 4 entries atomically for the correctness. +int ObMultiVersionGarbageCollector::study() +{ + int ret = OB_SUCCESS; + share::SCN min_unallocated_GTS(share::SCN::max_scn()); + share::SCN min_unallocated_WRS(share::SCN::max_scn()); + share::SCN max_committed_txn_version(share::SCN::max_scn()); + share::SCN min_active_txn_version(share::SCN::max_scn()); + + ObTimeGuard timeguard(__func__, 1 * 1000 * 1000); + + // standby cluster uses the same interface for GTS + if (OB_FAIL(study_min_unallocated_GTS(min_unallocated_GTS))) { + MVCC_LOG(WARN, "study min unallocated GTS failed", K(ret)); + } else if (!min_unallocated_GTS.is_valid() + || min_unallocated_GTS.is_min() + || min_unallocated_GTS.is_max()) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "wrong min unallocated GTS", + K(ret), K(min_unallocated_GTS), KPC(this)); + } else { + MVCC_LOG(INFO, "study min unallocated gts succeed", + K(ret), K(min_unallocated_GTS), KPC(this)); + } + + timeguard.click("study_min_unallocated_GTS"); + + if (OB_SUCC(ret)) { + if (!GCTX.is_standby_cluster() && // standby cluster does not support WRS + OB_FAIL(study_min_unallocated_WRS(min_unallocated_WRS))) { + MVCC_LOG(WARN, "study min unallocated GTS failed", K(ret)); + } else if (!min_unallocated_WRS.is_valid() + || min_unallocated_WRS.is_min() + || min_unallocated_WRS.is_max()) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "wrong min unallocated WRS", + K(ret), K(min_unallocated_WRS), KPC(this)); + } else { + MVCC_LOG(INFO, "study min unallocated wrs succeed", + K(ret), K(min_unallocated_WRS), KPC(this)); + } + } + + timeguard.click("study_min_unallocated_WRS"); + + if (OB_SUCC(ret)) { + if (OB_FAIL(study_max_committed_txn_version(max_committed_txn_version))) { + MVCC_LOG(WARN, "study max committed txn version failed", K(ret)); + } else if (!max_committed_txn_version.is_valid() + || max_committed_txn_version.is_max()) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "wrong max committed txn version", + K(ret), K(max_committed_txn_version), KPC(this)); + } else { + MVCC_LOG(INFO, "study max committed txn version succeed", + K(ret), K(max_committed_txn_version), KPC(this)); + } + } + + timeguard.click("study_max_commited_txn_version"); + + if (OB_SUCC(ret)) { + if (OB_FAIL(study_min_active_txn_version(min_active_txn_version))) { + MVCC_LOG(WARN, "study min active txn version failed", K(ret)); + } else { + MVCC_LOG(INFO, "study min active txn version succeed", + K(ret), K(min_active_txn_version), KPC(this)); + } + } + + timeguard.click("study_min_active_txn_version"); + + if (OB_SUCC(ret) && + can_report() && + OB_FAIL(report(min_unallocated_GTS, + min_unallocated_WRS, + max_committed_txn_version, + min_active_txn_version))) { + MVCC_LOG(WARN, "report garbage collect info failed", K(ret)); + } + + timeguard.click("report"); + + MVCC_LOG(INFO, "study multi version garabage collector end", + K(ret), KPC(this), K(min_unallocated_GTS), K(min_unallocated_GTS), + K(max_committed_txn_version), K(min_active_txn_version)); + + return ret; +} + +// The read snapshot version may base on GTS for most txns, so we need study it on each machine. +int ObMultiVersionGarbageCollector::study_min_unallocated_GTS(share::SCN &min_unallocated_GTS) +{ + int ret = OB_SUCCESS; + + const transaction::MonotonicTs stc_ahead = transaction::MonotonicTs::current_time() ; + transaction::MonotonicTs unused_receive_gts_ts(0); + const int64_t timeout_us = 1 * 1000 * 1000; // 1s + const int64_t expire_time_us = common::ObTimeUtility::current_time() + timeout_us; + share::SCN gts_scn; + + do { + // We get the gts each 10ms in order not to report too much error during reboot + // and continue to fetch under the failure among 1s. + ret = OB_TS_MGR.get_gts(MTL_ID(), + stc_ahead, + NULL, // gts task + gts_scn, + unused_receive_gts_ts); + if (ret == OB_EAGAIN) { + if (common::ObTimeUtility::current_time() > expire_time_us) { + ret = OB_TIMEOUT; + } else { + ob_usleep(10 * 1000/*10ms*/); + } + } else if (OB_FAIL(ret)) { + MVCC_LOG(WARN, "get gts fail", KR(ret)); + } else if (!gts_scn.is_valid()) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "get gts fail", K(gts_scn), K(ret)); + } else { + min_unallocated_GTS = gts_scn; + } + } while (ret == OB_EAGAIN); + + return ret; +} + +// The read snapshot version may base on WRS for the boundary weak read txn, so we +// need study it on each machine. +int ObMultiVersionGarbageCollector::study_min_unallocated_WRS( + share::SCN &min_unallocated_WRS) +{ + int ret = OB_SUCCESS; + + if (OB_FAIL(MTL(transaction::ObTransService*)->get_weak_read_snapshot_version( + transaction::ObWeakReadUtil::max_stale_time_for_weak_consistency(MTL_ID()), + min_unallocated_WRS))) { + MVCC_LOG(WARN, "fail to get weak read snapshot", K(ret)); + } + + return ret; +} + +// The read snapshot version may base on max committed txn version for the +// single ls txn, so we need study it on each machine. +int ObMultiVersionGarbageCollector::study_max_committed_txn_version( + share::SCN &max_committed_txn_version) +{ + int ret = OB_SUCCESS; + + max_committed_txn_version = MTL(transaction::ObTransService*)-> + get_tx_version_mgr().get_max_commit_ts(false/*elr*/); + + if (max_committed_txn_version.is_base_scn()) { + // if the max committed txn version is base_scn(not updated by any txns and + // async loop worker), we need ignore it and retry the next time + ret = OB_EAGAIN; + MVCC_LOG(WARN, "get max committed txn version is base version", + K(ret), K(max_committed_txn_version)); + } + + return ret; +} + +// We need collect all active txns, so decide to collect all snapshot version on +// one machine through tranversing the sessions. Lets' show all possibilities of +// the txns: +// 1. RR/SI, AC=0 txn: it will create the session with tx_desc on the scheduler +// and record the snapshot_version on it. We can directly use it. +// 2. RC, AC=0 txn: it will create the session with tx_desc on the scheduler +// while not recording the snapshot_version. We currently use session_state +// and query_start_ts to act as the alive stmt snapshot. +// TODO(handora.qc): record the snapshot version to tx_desc in the feture. +// 3. AC=1 txn: it may contain no tx_desc on session, while it must create session. +// Even for remote execution, it will create session on the execution machine. +int ObMultiVersionGarbageCollector::study_min_active_txn_version( + share::SCN &min_active_txn_version) +{ + int ret = OB_SUCCESS; + + if (OB_ISNULL(GCTX.session_mgr_)) { + ret = OB_INVALID_ARGUMENT; + MVCC_LOG(WARN, "session mgr is nullptr"); + } else if (OB_FAIL(GCTX.session_mgr_-> + get_min_active_snapshot_version(min_active_txn_version))) { + MVCC_LOG(WARN, "get min active snaphot version failed", K(ret)); + } + + return ret; +} + +// refresh_ will timely collect the multi-version garbage collection info from +// inner table. We need take three factors into consideration. 1, GC may be +// disabled by disk monitor; 2, gc info may go back, and we should ignore these +// value; 3, gc may use INT64_MAX under exceptional condition, and we also need +// ignore it. +int ObMultiVersionGarbageCollector::refresh_() +{ + int ret = OB_SUCCESS; + concurrency_control::ObMultiVersionGCSnapshotCalculator collector; + + ObTimeGuard timeguard(__func__, 1 * 1000 * 1000); + + if (is_refresh_fail()) { + ret = OB_EAGAIN; + MVCC_LOG(WARN, "mock refresh failed", K(ret), KPC(this), K(collector)); + } else if (OB_FAIL(MTL(concurrency_control::ObMultiVersionGarbageCollector *)->collect(collector))) { + MVCC_LOG(WARN, "collect snapshot info sql failed", K(ret), KPC(this), K(collector)); + } else { + // Step1: check whether gc status is disabled, then set or reset the gc + // status based on the collector's result; + decide_gc_status_(collector.get_status()); + timeguard.click("decide_gc_status_"); + + // Step2: whether gc status is wrong or not on the server, we need refresh + // it continuously. We will ignore the return code because it will not + // effect the refresh result. + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(disk_monitor_(collector.is_this_server_disabled()))) { + MVCC_LOG(WARN, "disk mintor failed", KPC(this), K(collector), K(tmp_ret)); + } + + timeguard.click("disk_mointor_"); + + // Step3: cache the reserved snapshot of active txn for future use. + // NB: be care of the lower value and maximum value which is not reasonable + decide_reserved_snapshot_version_(collector.get_reserved_snapshot_version()); + + timeguard.click("decide_reserved_snapshot_"); + + MVCC_LOG(INFO, "multi-version garbage collector refresh successfully", + KPC(this), K(collector)); + } + + return ret; +} + +void ObMultiVersionGarbageCollector::decide_gc_status_(const ObMultiVersionGCStatus gc_status) +{ + if (gc_status & ObMultiVersionGCStatus::DISABLED_GC_STATUS) { + MVCC_LOG(WARN, "gc status is disabled", KPC(this), + K(global_reserved_snapshot_), K(gc_status)); + gc_is_disabled_ = true; + } else if (gc_is_disabled_) { + MVCC_LOG(INFO, "gc status is enabled", KPC(this), + K(global_reserved_snapshot_), K(gc_status)); + gc_is_disabled_ = false; + } +} + +void ObMultiVersionGarbageCollector::decide_reserved_snapshot_version_( + const share::SCN reserved_snapshot) +{ + int ret = OB_SUCCESS; + + if (!reserved_snapshot.is_valid()) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "reserved is not valid", K(ret), KPC(this), + K(global_reserved_snapshot_), K(reserved_snapshot)); + } else if (reserved_snapshot < global_reserved_snapshot_) { + if ((global_reserved_snapshot_.get_val_for_tx() - + reserved_snapshot.get_val_for_tx()) / 1000 > 30 * 1_min) { + // We ignore the reserved snapshot with too late snapshot and report + // error to maintain normal processes + MVCC_LOG(ERROR, "update a too smaller reserved snapshot", K(ret), KPC(this), + K(global_reserved_snapshot_), K(reserved_snapshot)); + } else { + MVCC_LOG(WARN, "update a smaller reserved snapshot", K(ret), KPC(this), + K(global_reserved_snapshot_), K(reserved_snapshot)); + } + } else if (reserved_snapshot.is_max()) { + MVCC_LOG(WARN, "reserved snapshot is max value", K(ret), KPC(this), + K(global_reserved_snapshot_), K(reserved_snapshot)); + } else { + MVCC_LOG(INFO, "succeed to update global reserved snapshot", K(ret), KPC(this), + K(global_reserved_snapshot_), K(reserved_snapshot)); + global_reserved_snapshot_.atomic_set(reserved_snapshot); + } +} + +share::SCN ObMultiVersionGarbageCollector::get_reserved_snapshot_for_active_txn() const +{ + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + + if (!tenant_config->_mvcc_gc_using_min_txn_snapshot) { + return share::SCN::max_scn(); + } else if (refresh_error_too_long_) { + if (REACH_TENANT_TIME_INTERVAL(1_s)) { + MVCC_LOG(WARN, "get reserved snapshot for active txn with long not updated", KPC(this)); + } + return share::SCN::max_scn(); + } else if (gc_is_disabled_) { + if (REACH_TENANT_TIME_INTERVAL(1_s)) { + MVCC_LOG(WARN, "get reserved snapshot for active txn with gc is disabled", KPC(this)); + } + return share::SCN::max_scn(); + } else { + return global_reserved_snapshot_.atomic_load(); + } +} + +bool ObMultiVersionGarbageCollector::is_gc_disabled() const +{ + return gc_is_disabled_ // gc status is not allowed + || refresh_error_too_long_; // refresh inner table failed +} + +// collect reads all entries from inner-table, and apply functor to all entries +// to let users use it freely. +// NB: it will stop if functor report error, so use it carefully +int ObMultiVersionGarbageCollector::collect(ObMultiVersionGCSnapshotFunctor& calculator) +{ + int ret = OB_SUCCESS; + ObSqlString sql; + + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + common::sqlclient::ObMySQLResult *result = NULL; + const uint64_t tenant_id = MTL_ID(); + const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id); + + if (OB_ISNULL(GCTX.sql_proxy_)) { + ret = OB_NOT_INIT; + MVCC_LOG(WARN, "sql_proxy_ not init yet, collect abort", KR(ret)); + } else if (OB_FAIL(sql.assign_fmt(QUERY_ALL_RESERVED_SNAPSHOT_SQL, + share::OB_ALL_RESERVED_SNAPSHOT_TNAME, + tenant_id))) { + MVCC_LOG(WARN, "generate QUERY_ALL_SNAPSHOT_SQL fail", KR(ret)); + } else if (OB_FAIL(GCTX.sql_proxy_->read(res, meta_tenant_id, sql.ptr()))) { + MVCC_LOG(WARN, "execute sql read fail", KR(ret), K(meta_tenant_id), K(tenant_id), K(sql)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "execute sql fail", KR(ret), K(tenant_id), K(meta_tenant_id), K(sql)); + } else { + int64_t snapshot_version = 0; + share::SCN snapshot_version_scn; + ObMultiVersionSnapshotType snapshot_type = ObMultiVersionSnapshotType::MIN_SNAPSHOT_TYPE; + ObMultiVersionGCStatus gc_status = ObMultiVersionGCStatus::INVALID_GC_STATUS; + int64_t create_time = 0; + char svr_ip_buf[MAX_IP_ADDR_LENGTH + 1] = {0}; + int64_t svr_ip_len = 0; + uint64_t svr_port = 0; + ObAddr addr; + + while (OB_SUCC(ret) && OB_SUCC(result->next())) { + EXTRACT_UINT_FIELD_MYSQL(*result, "snapshot_version", snapshot_version, int64_t); + EXTRACT_UINT_FIELD_MYSQL(*result, "snapshot_type", snapshot_type, concurrency_control::ObMultiVersionSnapshotType); + EXTRACT_UINT_FIELD_MYSQL(*result, "create_time", create_time, int64_t); + EXTRACT_UINT_FIELD_MYSQL(*result, "status", gc_status, concurrency_control::ObMultiVersionGCStatus); + EXTRACT_STRBUF_FIELD_MYSQL(*result, "svr_ip", svr_ip_buf, 128, svr_ip_len); + EXTRACT_UINT_FIELD_MYSQL(*result, "svr_port", svr_port, uint64_t); + + if (!addr.set_ip_addr(svr_ip_buf, svr_port)) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(WARN, "set svr addr failed", K(svr_ip_buf), K(svr_port)); + } else if (OB_FAIL(snapshot_version_scn.convert_for_inner_table_field(snapshot_version))) { + MVCC_LOG(WARN, "set min snapshot version scn failed", K(ret), K(snapshot_version)); + } else if (OB_FAIL(calculator(snapshot_version_scn, + snapshot_type, + gc_status, + create_time, + addr))) { + MVCC_LOG(WARN, "calculate snapshot version failed", K(ret)); + } else { + MVCC_LOG(INFO, "multi version garbage colloector collects successfully", K(sql), + K(snapshot_version), K(snapshot_type), K(create_time), K(addr), K(gc_status)); + } + } + + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } + } + } + + return ret; +} + +// mock for disabling report +bool ObMultiVersionGarbageCollector::can_report() +{ + return true; +} + +// mock for disabling refresh +bool ObMultiVersionGarbageCollector::is_refresh_fail() +{ + return false; +} + +// report will report the four entries into the inner table. +// NB: the 4 entries must be inserted atomically as the reason has been talked +// about in the function 'study' for the rule 2. +int ObMultiVersionGarbageCollector::report(const share::SCN min_unallocated_GTS, + const share::SCN min_unallocated_WRS, + const share::SCN max_committed_txn_version, + const share::SCN min_active_txn_version) +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + const ObAddr &self_addr = GCTX.self_addr(); + char ip_buffer[MAX_IP_ADDR_LENGTH + 1] = {0}; + const uint64_t tenant_id = MTL_ID(); + const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id); + const int64_t current_ts = ObClockGenerator::getRealClock(); + + if (OB_UNLIKELY(!self_addr.ip_to_string(ip_buffer, MAX_IP_ADDR_LENGTH))) { + ret = OB_INVALID_ARGUMENT; + MVCC_LOG(WARN, "ip to string failed", K(self_addr)); + } else if (OB_FAIL(sql.assign_fmt(INSERT_ON_UPDATE_ALL_RESERVED_SNAPSHOT_SQL, + share::OB_ALL_RESERVED_SNAPSHOT_TNAME, + // entries of the MIN_UNALLOCATED_GTS + tenant_id, + (uint64_t)(ObMultiVersionSnapshotType::MIN_UNALLOCATED_GTS), + int(MAX_IP_ADDR_LENGTH), ip_buffer, + self_addr.get_port(), + current_ts, + (uint64_t)(ObMultiVersionGCStatus::NORMAL_GC_STATUS), + min_unallocated_GTS.get_val_for_inner_table_field(), + // entries of the MIN_UNALLOCATED_WRS + tenant_id, + (uint64_t)(ObMultiVersionSnapshotType::MIN_UNALLOCATED_WRS), + int(MAX_IP_ADDR_LENGTH), ip_buffer, + self_addr.get_port(), + current_ts, + (uint64_t)(ObMultiVersionGCStatus::NORMAL_GC_STATUS), + min_unallocated_WRS.get_val_for_inner_table_field(), + // entries of the MAX_COMMITTED_TXN_VERSION + tenant_id, + (uint64_t)(ObMultiVersionSnapshotType::MAX_COMMITTED_TXN_VERSION), + int(MAX_IP_ADDR_LENGTH), ip_buffer, + self_addr.get_port(), + current_ts, + (uint64_t)(ObMultiVersionGCStatus::NORMAL_GC_STATUS), + max_committed_txn_version.get_val_for_inner_table_field(), + // entries of the ACTIVE_TXN_SNAPSHOT + tenant_id, + (uint64_t)(ObMultiVersionSnapshotType::ACTIVE_TXN_SNAPSHOT), + int(MAX_IP_ADDR_LENGTH), ip_buffer, + self_addr.get_port(), + current_ts, + (uint64_t)(ObMultiVersionGCStatus::NORMAL_GC_STATUS), + min_active_txn_version.get_val_for_inner_table_field()))) { + MVCC_LOG(WARN, "format sql fail", KR(ret), K(sql)); + } else if (OB_ISNULL(GCTX.sql_proxy_)) { + MVCC_LOG(WARN, "sql_proxy_ not init yet, report abort", KR(ret), K(sql)); + } else if (OB_FAIL(GCTX.sql_proxy_->write(meta_tenant_id, sql.ptr(), affected_rows))) { + MVCC_LOG(WARN, "execute sql fail", KR(ret), K(sql)); + } else if (8 != affected_rows && // for on duplicate update + 4 != affected_rows) { // for first insert + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "report multi version snapshot failed", KR(ret), K(sql), K(affected_rows)); + } else { + MVCC_LOG(INFO, "report multi version snapshot success", KR(ret), K(sql), K(affected_rows)); + } + + return ret; +} + +// update_status will update the four entries' status into the inner table. +// NB: the 4 entries must be updated atomically as the reason has been talked +// about in the function 'study' for the rule 2. +int ObMultiVersionGarbageCollector::update_status(const ObMultiVersionGCStatus status) +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + const ObAddr &self_addr = GCTX.self_addr(); + char ip_buffer[MAX_IP_ADDR_LENGTH + 1] = {0}; + const uint64_t tenant_id = MTL_ID(); + const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id); + + if (OB_UNLIKELY(!self_addr.ip_to_string(ip_buffer, MAX_IP_ADDR_LENGTH))) { + ret = OB_INVALID_ARGUMENT; + MVCC_LOG(WARN, "ip to string failed", K(self_addr)); + } else if (OB_FAIL(sql.assign_fmt(UPDATE_RESERVED_SNAPSHOT_STATUS, + share::OB_ALL_RESERVED_SNAPSHOT_TNAME, + (uint64_t)(status), + tenant_id, + int(MAX_IP_ADDR_LENGTH), ip_buffer, + self_addr.get_port()))) { + MVCC_LOG(WARN, "format sql fail", KR(ret), K(sql)); + } else if (OB_ISNULL(GCTX.sql_proxy_)) { + MVCC_LOG(WARN, "sql_proxy_ not init yet, report abort", KR(ret), K(sql)); + } else if (OB_FAIL(GCTX.sql_proxy_->write(meta_tenant_id, sql.ptr(), affected_rows))) { + MVCC_LOG(WARN, "execute sql fail", KR(ret), K(sql)); + } else if (0 != affected_rows && // update with the same row + 4 != affected_rows) { // normal update succeed + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "report multi version snapshot failed", KR(ret), K(sql), K(affected_rows)); + } else { + MVCC_LOG(INFO, "update multi version snapshot success", KR(ret), K(sql), K(affected_rows)); + } + + return ret; +} + +// We need reclaim expired entries in the inner table under exceptional +// conditions. For example, when the node where the active txn is located +// cannot update the inner table in time due to abnormal reasons, the snapshot +// value cannot be advanced and a large number of versions cannot be recycled. +// So we need to handle snapshot processing in abnormal situations. +// +// Firstly, we rely on the sys ls's ability to manage the ls. We hope that if +// the node cannot contact to the inner table due to abnormalities, then +// eventually the entries of this node will be removed from the innner table, so +// that we no longer rely on this Timestamps provided by the node as recycling +// snapshots.(implemented in the reclaim) +// +// Secondly, we rely on the each node's ability to ignore faulty timestamps. We +// hope that if the sys ls cannot contact to the inner table and remove the +// entries in time, each node will ignore the timestamps that has not been +// updated for a long time.(implemented in the ObMultiVersionGCSnapshotCalculator). +// +// Finally, in the worst case, the node cannot contact to the inner table and +// fail to advance the snapshot it maintained. We think the multi-version data +// on the node is not reachable for active txns, so we ignore the value of the +// inner table and use the undo_retention and the snapshot_gc_ts as the new +// mechanism.(implemented in the repeat_refresh) +// +// All in all, our purpose is that no matter what the exception is, we must be +// able to provide a user-reasonable(may be value of the customer) recycling +// snapshot that all users can understand. +int ObMultiVersionGarbageCollector::reclaim() +{ + int ret = OB_SUCCESS; + ObLS *ls = NULL; + storage::ObLSHandle ls_handle; + logservice::ObLogHandler *handler = nullptr; + int64_t old_proposal_id = 0; + ObArray reclaimable_servers; + ObArray snapshot_servers; + bool is_this_server_disabled = false; + common::ObRole role; + + ObTimeGuard timeguard(__func__, 1 * 1000 * 1000); + + if (OB_FAIL(MTL(storage::ObLSService*)-> + get_ls(share::SYS_LS, ls_handle, ObLSGetMod::MULTI_VERSION_GARBAGE_COLLOECTOR_MOD))) { + MVCC_LOG(WARN, "get sys ls failed", K(ret)); + ret = OB_SUCCESS; + } else if (OB_ISNULL(ls = ls_handle.get_ls()) + || OB_ISNULL(handler = ls_handle.get_ls()->get_log_handler())) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(ERROR, "log stream is NULL", K(ret), K(ls)); + } else if (OB_FAIL(handler->get_role(role, old_proposal_id))) { + MVCC_LOG(WARN, "fail to get role", KR(ret)); + } else if (common::is_leader_like(role)) { + timeguard.click("get_leader"); + + // TODO(handora.qc): use nicer timer + const int64_t current_timestamp = ObClockGenerator::getRealClock(); + + ObMultiVersionGCSnapshotOperator collector( + [current_timestamp, + &reclaimable_servers, + &snapshot_servers, + &is_this_server_disabled](const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) -> int { + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + bool need_reclaim = false; + + if (OB_FAIL(snapshot_servers.push_back(addr))) { + MVCC_LOG(WARN, "push array failed", K(ret)); + } else { + // TODO(handora.qc): use a better time monitor for the node lost for a long time + if (current_timestamp > create_time + && current_timestamp - create_time > GARBAGE_COLLECT_RECLAIM_DURATION) { + bool is_exist = true; + if (OB_TMP_FAIL(share::ObAllServerTracer::get_instance().is_server_exist(addr, is_exist))) { + MVCC_LOG(WARN, "check all server tracer failed", K(tmp_ret)); + } else if (is_exist) { + // Case 1: server exists, while not renew snapshot for a long time + bool is_alive = false; + if (OB_TMP_FAIL(share::ObAllServerTracer::get_instance().check_server_alive(addr, is_alive))) { + MVCC_LOG(WARN, "check all server tracer failed", K(tmp_ret)); + } else if (is_alive) { + // Case 1.1: server is alive, we report the error for not renewing. + // monitor should asserts the exception + MVCC_LOG(ERROR, "server alives while not renew for a long time", K(create_time), + K(current_timestamp), K(addr), K(snapshot_type), K(snapshot_version)); + need_reclaim = true; + } else { + // Case 1.2: server is not alive, we report the WARN and reclaim + // it immediately + MVCC_LOG(WARN, "server not alives while not renew for a long time", K(create_time), + K(current_timestamp), K(addr), K(snapshot_type), K(snapshot_version)); + need_reclaim = true; + } + } else { + // Case 2: server doesnot exits, we report the WARN and reclaim + // it immediately + MVCC_LOG(WARN, "server doesnot exists so we should remove it", K(create_time), + K(current_timestamp), K(addr), K(snapshot_type), K(snapshot_version)); + need_reclaim = true; + } + } + + if (need_reclaim) { + bool exist = false; + for (int64_t i = 0; !exist && i < reclaimable_servers.count(); i++) { + if (addr == reclaimable_servers[i]) { + exist = true; + } + } + if (!exist && OB_FAIL(reclaimable_servers.push_back(addr))) { + MVCC_LOG(WARN, "push back array failed", K(ret)); + } + } + } + + if (!is_this_server_disabled && + addr == GCTX.self_addr() && + status != ObMultiVersionGCStatus::NORMAL_GC_STATUS) { + is_this_server_disabled = true; + } + + return ret; + }); + + // collect all info for reclaimable servers and all reported servers + if (OB_FAIL(collect(collector))) { + MVCC_LOG(WARN, "collect snapshot info failed", K(ret)); + } else { + int tmp_ret = OB_SUCCESS; + timeguard.click("collect"); + + if (0 == reclaimable_servers.count()) { + // all snapshot info is not reclaimable + MVCC_LOG(INFO, "skip all alivavle snapshots info"); + // reclaim all uncessary servers + } else if (OB_TMP_FAIL((reclaim_(reclaimable_servers)))) { + MVCC_LOG(WARN, "reclaim snapshot info failed", K(tmp_ret), + K(reclaimable_servers)); + if (OB_SUCC(ret)) { + ret = tmp_ret; + } + } + + timeguard.click("reclaim_"); + + if (0 == snapshot_servers.count()) { + MVCC_LOG(WARN, "no alive servers now, please check it clearly", + KPC(this), K(snapshot_servers), K(reclaimable_servers)); + // monitor all servers + } else if (OB_TMP_FAIL(monitor_(snapshot_servers))) { + MVCC_LOG(WARN, "snapshots servers are mintor failed", + KPC(this), K(snapshot_servers), K(tmp_ret)); + if (OB_SUCC(ret)) { + ret = tmp_ret; + } + } + + timeguard.click("monitor_"); + } + } + + MVCC_LOG(INFO, "reclaim multi version garabage collector end", KPC(this), + K(ret), K(role), K(reclaimable_servers), K(snapshot_servers), + K(is_this_server_disabled)); + + return ret; +} + +// mointor checks all servers in the inner table and server manager and check +// whether there exists a server in the server manager and does not report its +// timestamps from beginning to the end. +int ObMultiVersionGarbageCollector::monitor_(const ObArray &snapshot_servers) +{ + int ret = OB_SUCCESS; + ObArray lost_servers; + + if (OB_FAIL(share::ObAllServerTracer::get_instance().for_each_server_status( + [&snapshot_servers, + &lost_servers](const share::ObServerStatus &status) -> int { + int ret = OB_SUCCESS; + bool found = false; + + // find servers that recorded in the server manger while has + // not reported its timestamp. + for (int64_t i = 0; !found && i < snapshot_servers.count(); ++i) { + if (status.server_ == snapshot_servers[i]) { + found = true; + } + } + + if (!found) {// not found in __all_reserved_snapshot inner table + if (OB_FAIL(lost_servers.push_back(status.server_))) { + MVCC_LOG(WARN, "lost servers push back failed", K(ret)); + } else if (!status.is_valid()) { + MVCC_LOG(ERROR, "invalid status", K(ret), K(status)); + // if not in service, we ignore it and report the warning + } else if (!status.in_service() || status.is_stopped()) { + MVCC_LOG(WARN, "server is not alive, we will remove soon", K(ret), K(status)); + // if not alive, we ignore it and report the warning + } else if (!status.is_alive()) { + MVCC_LOG(WARN, "server is not alive, please pay attention", K(ret), K(status)); + } else { + // may be lost or do not contain the tenant + // TODO(handora.qc): make it better and more clear + MVCC_LOG(INFO, "server is alive when mointor", K(ret), K(status)); + } + } + + return ret; + }))) { + MVCC_LOG(WARN, "for each server status failed", K(ret)); + } else { + MVCC_LOG(INFO, "garbage collector monitor server status monitor", + K(snapshot_servers), K(lost_servers)); + } + + return ret; +} + +// disk monitor will monitor the current disk status and report to inner table +// in time when finding the status of the two parties does not match. +// +// The demand comes from the following story: +// Some scenes often appear in the online environment that long-running +// snapshots prevent data from recycling, so we must take disk usage into +// consideration. So we timely check the disk usage and report it to the +// inner_table(called gc status). And all server will check the gc status +// before using it. +int ObMultiVersionGarbageCollector::disk_monitor_(const bool is_this_server_alomost_full) +{ + int ret = OB_SUCCESS; + bool is_almost_full = false; + bool need_report = false; + + if (OB_FAIL(is_disk_almost_full_(is_almost_full))) { + MVCC_LOG(WARN, "check disk almost full failed", K(ret), KPC(this)); + } else if (is_this_server_alomost_full && is_almost_full) { + need_report = false; + MVCC_LOG(WARN, "the disk still be full of the disk", K(ret), KPC(this)); + } else if (!is_this_server_alomost_full && is_almost_full) { + MVCC_LOG(WARN, "the disk becoming full of the disk", K(ret), KPC(this)); + need_report = true; + } else if (!is_this_server_alomost_full && !is_almost_full) { + // normal scense + need_report = false; + } else if (is_this_server_alomost_full && !is_almost_full) { + MVCC_LOG(INFO, "the disk becoming not full of the disk", K(ret), KPC(this)); + need_report = true; + } + + if (need_report) { + ObMultiVersionGCStatus status = is_almost_full ? + ObMultiVersionGCStatus::DISABLED_GC_STATUS : + ObMultiVersionGCStatus::NORMAL_GC_STATUS; + if (OB_FAIL(update_status(status))) { + MVCC_LOG(WARN, "disk monitor failed", K(ret), K(status)); + } else { + MVCC_LOG(INFO, "report disk monitor succeed", K(ret), K(status), + K(is_this_server_alomost_full), K(is_almost_full)); + } + } + + return ret; +} + +// reclaim remove entries according to reclaimable_snapshots_info +int ObMultiVersionGarbageCollector::reclaim_(const ObArray &reclaimable_servers) +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + char ip_buffer[MAX_IP_ADDR_LENGTH + 1] = {0}; + const uint64_t tenant_id = MTL_ID(); + const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id); + + for (int64_t i = 0; i < reclaimable_servers.count(); ++i) { + ObAddr addr = reclaimable_servers[i]; + if (OB_UNLIKELY(!addr.ip_to_string(ip_buffer, MAX_IP_ADDR_LENGTH))) { + ret = OB_INVALID_ARGUMENT; + MVCC_LOG(WARN, "ip to string failed", K(addr)); + } else if (OB_FAIL(sql.assign_fmt(DELETE_EXPIRED_RESERVED_SNAPSHOT, + share::OB_ALL_RESERVED_SNAPSHOT_TNAME, + tenant_id, + int(MAX_IP_ADDR_LENGTH), ip_buffer, + addr.get_port()))) { + MVCC_LOG(WARN, "format sql fail", KR(ret), K(sql)); + } else if (OB_ISNULL(GCTX.sql_proxy_)) { + MVCC_LOG(WARN, "sql_proxy_ not init yet, report abort", KR(ret), K(sql)); + } else if (OB_FAIL(GCTX.sql_proxy_->write(meta_tenant_id, sql.ptr(), affected_rows))) { + MVCC_LOG(WARN, "execute sql fail", KR(ret), K(sql)); + } else if (OB_UNLIKELY(0 != affected_rows && 4 != affected_rows)) { + ret = OB_ERR_UNEXPECTED; + MVCC_LOG(WARN, "affected rows is wrong", KR(ret), K(sql), K(affected_rows)); + } else { + MVCC_LOG(INFO, "reclaim expired multi version snapshot success", + KR(ret), K(sql), K(addr)); + } + } + + return ret; +} + +// Some scenes often appear in the online environment that long-running +// snapshots prevent data from recycling, so we must take disk usage into +// consideration. So we timely check the disk usage and report it to the +// inner_table(called gc status). And all server will check the gc status +// before using it. +int ObMultiVersionGarbageCollector::is_disk_almost_full_(bool &is_almost_full) +{ + int ret = OB_SUCCESS; + is_almost_full = false; + const int64_t required_size = 0; + + if (OB_FAIL(THE_IO_DEVICE->check_space_full(required_size))) { + if (OB_SERVER_OUTOF_DISK_SPACE == ret) { + ret = OB_SUCCESS; + is_almost_full = true; + MVCC_LOG(WARN, "disk is almost full, we should give up", KPC(this)); + } else { + MVCC_LOG(WARN, "failed to check space full", K(ret)); + } + } + + return ret; +} + +ObMultiVersionGCSnapshotCalculator::ObMultiVersionGCSnapshotCalculator() + : reserved_snapshot_version_(share::SCN::max_scn()), + reserved_snapshot_type_(ObMultiVersionSnapshotType::MIN_SNAPSHOT_TYPE), + reserved_status_(ObMultiVersionGCStatus::INVALID_GC_STATUS), + reserved_create_time_(0), + reserved_addr_(), + is_this_server_disabled_(false), + status_(ObMultiVersionGCStatus::NORMAL_GC_STATUS) {} + +ObMultiVersionGCSnapshotCalculator::~ObMultiVersionGCSnapshotCalculator() +{ + reserved_snapshot_version_ = share::SCN::max_scn(); + reserved_snapshot_type_ = ObMultiVersionSnapshotType::MIN_SNAPSHOT_TYPE; + reserved_status_ = ObMultiVersionGCStatus::INVALID_GC_STATUS; + reserved_create_time_ = 0; + reserved_addr_.reset(); + is_this_server_disabled_ = false; + status_ = ObMultiVersionGCStatus::NORMAL_GC_STATUS; +} + +int ObMultiVersionGCSnapshotCalculator::operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) +{ + int ret = OB_SUCCESS; + // TODO(handora.qc): change machine time to nicer time + const int64_t current_ts = ObClockGenerator::getRealClock(); + + // Step1: calculate the minumium reserved version and record it + if (snapshot_version < reserved_snapshot_version_) { + if (current_ts > create_time && + current_ts - create_time > 2 * ObMultiVersionGarbageCollector::GARBAGE_COLLECT_RECLAIM_DURATION && + // for mock or test that change GARBAGE_COLLECT_EXEC_INTERVAL to a small value + current_ts - create_time > 2 * 3 * 10_min) { + // we report error here because we may relay on the reclaim strategy of + // garbage collector to reclaim the data of died machine + MVCC_LOG(ERROR, "ignore too old version", K(snapshot_version), + K(snapshot_type), K(current_ts), K(create_time), K(addr)); + } else { + reserved_snapshot_version_ = snapshot_version; + reserved_snapshot_type_ = snapshot_type; + reserved_create_time_ = create_time; + reserved_addr_ = addr; + } + } + + // Step2: ensure the gc status of the current server + if (!is_this_server_disabled_ && + addr == GCTX.self_addr() && + status != ObMultiVersionGCStatus::NORMAL_GC_STATUS) { + is_this_server_disabled_ = true; + } + + // Step3: merge the status of all multi-version gc status + status_ = status_ | status; + + return ret; +} + +share::SCN ObMultiVersionGCSnapshotCalculator::get_reserved_snapshot_version() const +{ + return reserved_snapshot_version_; +} + +ObMultiVersionGCStatus ObMultiVersionGCSnapshotCalculator::get_status() const +{ + return status_; +} + +ObMultiVersionSnapshotInfo::ObMultiVersionSnapshotInfo() + : snapshot_version_(share::SCN::min_scn()), + snapshot_type_(ObMultiVersionSnapshotType::MIN_SNAPSHOT_TYPE), + status_(ObMultiVersionGCStatus::INVALID_GC_STATUS), + create_time_(0), + addr_() {} + +ObMultiVersionSnapshotInfo::ObMultiVersionSnapshotInfo(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) + : snapshot_version_(snapshot_version), + snapshot_type_(snapshot_type), + status_(status), + create_time_(create_time), + addr_(addr) {} + +ObMultiVersionGCSnapshotCollector::ObMultiVersionGCSnapshotCollector( + ObIArray &snapshot_info) + : snapshots_info_(snapshot_info) {} + +ObMultiVersionGCSnapshotCollector::~ObMultiVersionGCSnapshotCollector() {} + +int ObMultiVersionGCSnapshotCollector::operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) +{ + int ret = OB_SUCCESS; + + if (OB_FAIL(snapshots_info_.push_back(ObMultiVersionSnapshotInfo(snapshot_version, + snapshot_type, + status, + create_time, + addr)))) { + MVCC_LOG(WARN, "push back to snapshots info failed", K(ret)); + } + + return ret; +} + +ObMultiVersionGCSnapshotOperator::ObMultiVersionGCSnapshotOperator( + const ObMultiVersionGCSnapshotFunction &func) + : func_(func) {} + +int ObMultiVersionGCSnapshotOperator::operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) +{ + return func_(snapshot_version, snapshot_type, status, create_time, addr); +} + +// Functor to fetch the status of all alived session +// TODO(handora.qc): using better timestamp instead of query_start_time +bool GetMinActiveSnapshotVersionFunctor::operator()(sql::ObSQLSessionMgr::Key key, + sql::ObSQLSessionInfo *sess_info) +{ + int ret = OB_SUCCESS; + UNUSED(key); + + if (OB_ISNULL(sess_info)) { + ret = OB_NOT_INIT; + MVCC_LOG(WARN, "session info is NULL"); + } else if (false == sess_info->is_valid()) { + ret = OB_INVALID_ARGUMENT; + MVCC_LOG(WARN, "session info is not valid", K(ret)); + } else if (MTL_ID() == sess_info->get_effective_tenant_id()) { + sql::ObSQLSessionInfo::LockGuard data_lock_guard(sess_info->get_thread_data_lock()); + share::SCN snapshot_version(share::SCN::max_scn()); + + if (sess_info->is_in_transaction()) { + transaction::ObTxDesc *tx_desc = nullptr; + if (OB_ISNULL(tx_desc = sess_info->get_tx_desc())) { + MVCC_LOG(WARN, "tx desc is nullptr", KPC(sess_info)); + } else if (transaction::ObTxIsolationLevel::SERIAL == tx_desc->get_isolation_level() || + transaction::ObTxIsolationLevel::RR == tx_desc->get_isolation_level()) { + // Case 1: RR/SI with tx desc exists, so we can get snapshot version from tx desc + share::SCN tmp_snapshot_version = tx_desc->get_snapshot_version(); + if (tmp_snapshot_version.is_valid()) { + snapshot_version = tmp_snapshot_version; + } + MVCC_LOG(DEBUG, "RR/SI txn with tx_desc", K(MTL_ID()), KPC(tx_desc), KPC(sess_info), + K(snapshot_version), K(min_active_snapshot_version_)); + } else if (transaction::ObTxIsolationLevel::RC == tx_desc->get_isolation_level()) { + // Case 2: RC with tx desc exists, while the snapshot version is not + // maintained, so we use query start time instead + // TODO(handora.qc): use better snapshot version + if (sql::ObSQLSessionState::QUERY_ACTIVE == sess_info->get_session_state()) { + snapshot_version.convert_from_ts(sess_info->get_query_start_time()); + } + MVCC_LOG(DEBUG, "RC txn with tx_desc", K(MTL_ID()), KPC(tx_desc), KPC(sess_info), + K(snapshot_version), K(min_active_snapshot_version_)); + } else { + MVCC_LOG(WARN, "unknown txn with tx_desc", K(MTL_ID()), KPC(tx_desc), KPC(sess_info), + K(snapshot_version), K(min_active_snapshot_version_)); + } + } else { + if (transaction::ObTxIsolationLevel::SERIAL == sess_info->get_tx_isolation() || + transaction::ObTxIsolationLevel::RR == sess_info->get_tx_isolation()) { + // Case 3: RR/SI with tx desc does not exist, it is not for the scheduler + MVCC_LOG(DEBUG, "RR/SI txn with non tx_desc", K(MTL_ID()), KPC(sess_info), + K(snapshot_version), K(min_active_snapshot_version_)); + } else if (transaction::ObTxIsolationLevel::RC == sess_info->get_tx_isolation()) { + // Case 4: RC with tx desc does not exist, while the snapshot version is not + // maintained, so we use query start time instead + // TODO(handora.qc): use better snapshot version + if (sql::ObSQLSessionState::QUERY_ACTIVE == sess_info->get_session_state()) { + snapshot_version.convert_from_ts(sess_info->get_query_start_time()); + } + MVCC_LOG(DEBUG, "RC txn with non tx_desc", K(MTL_ID()), KPC(sess_info), + K(snapshot_version), K(min_active_snapshot_version_)); + } else { + MVCC_LOG(INFO, "unknown txn with non tx_desc", K(MTL_ID()), KPC(sess_info), + K(snapshot_version), K(min_active_snapshot_version_)); + } + } + + if (OB_SUCC(ret) && snapshot_version < min_active_snapshot_version_) { + min_active_snapshot_version_ = snapshot_version; + } + } + + return OB_SUCCESS == ret; +} + +} // namespace concurrency_control +} // namespace oceanbase diff --git a/src/storage/concurrency_control/ob_multi_version_garbage_collector.h b/src/storage/concurrency_control/ob_multi_version_garbage_collector.h new file mode 100644 index 000000000..8cc89a57b --- /dev/null +++ b/src/storage/concurrency_control/ob_multi_version_garbage_collector.h @@ -0,0 +1,374 @@ +/** + * 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_STORAGE_CONCURRENCY_CONTROL_OB_MULTI_VERSION_GARBAGE_COLLECTOR +#define OCEANBASE_STORAGE_CONCURRENCY_CONTROL_OB_MULTI_VERSION_GARBAGE_COLLECTOR + +#include "share/scn.h" +#include "storage/ls/ob_ls.h" +#include "lib/function/ob_function.h" +#include "share/ob_occam_timer.h" +#include "sql/session/ob_sql_session_info.h" +#include "sql/session/ob_sql_session_mgr.h" + +namespace oceanbase +{ +namespace concurrency_control +{ + +#define UPDATE_ONE_RESERVED_SNAPSHOT_SQL " \ + update %s set \ + snapshot_version = %ld, \ + create_time = %ld \ + where tenant_id = %lu \ + and snapshot_type = %lu \ + and svr_ip = '%.*s' \ + and svr_port = %d \ + " + +#define QUERY_ONE_RESERVED_SNAPSHOT_SQL_UDT " \ + select snapshot_version, snapshot_type, \ + svr_ip, svr_port, create_time \ + from %s \ + where tenant_id = %lu \ + and snapshot_type = %lu \ + and svr_ip = '%.*s' \ + and svr_port = %d \ + for update \ + " + +#define QUERY_ALL_RESERVED_SNAPSHOT_SQL " \ + select snapshot_version, snapshot_type, \ + svr_ip, svr_port, create_time, status \ + from %s \ + where tenant_id = %lu \ + " + +#define INSERT_ON_UPDATE_ALL_RESERVED_SNAPSHOT_SQL " \ + insert into %s \ + (tenant_id, snapshot_type, svr_ip, svr_port, \ + create_time, status, snapshot_version) \ + values (%lu, %lu, '%.*s', %d, '%ld', %ld, %ld), \ + (%lu, %lu, '%.*s', %d, '%ld', %ld, %ld), \ + (%lu, %lu, '%.*s', %d, '%ld', %ld, %ld), \ + (%lu, %lu, '%.*s', %d, '%ld', %ld, %ld) \ + on duplicate key update \ + create_time = VALUES(create_time), \ + snapshot_version = VALUES(snapshot_version) \ + " + +#define UPDATE_RESERVED_SNAPSHOT_STATUS " \ + update %s set \ + status = %ld \ + where tenant_id = %lu \ + and svr_ip = '%.*s' \ + and svr_port = %d \ + " + +#define QUERY_DISTINCT_SNAPSHOT_FOR_UPDATE " \ + select create_time, snapshot_type, \ + svr_ip, svr_port, create_time, status \ + from %s \ + where tenant_id = %lu \ + group by snapshot_type, svr_ip, svr_port \ + order by create_time desc \ + for update \ + " + +#define DELETE_EXPIRED_RESERVED_SNAPSHOT " \ + delete from %s \ + where tenant_id = %lu \ + and svr_ip = '%.*s' \ + and svr_port = %d \ + " + +enum ObMultiVersionSnapshotType : uint64_t +{ + MIN_SNAPSHOT_TYPE = 0, + MIN_UNALLOCATED_GTS = 1, + MIN_UNALLOCATED_WRS = 2, + MAX_COMMITTED_TXN_VERSION = 3, + ACTIVE_TXN_SNAPSHOT = 4, + MAX_SNAPSHOT_TYPE = 5, +}; + +enum ObMultiVersionGCStatus : uint64_t +{ + NORMAL_GC_STATUS = 0, + DISABLED_GC_STATUS = 1 << 0, + INVALID_GC_STATUS = UINT64_MAX, +}; + +inline ObMultiVersionGCStatus operator | (ObMultiVersionGCStatus a, ObMultiVersionGCStatus b) +{ + return static_cast(static_cast(a) | static_cast(b)); +} + +inline ObMultiVersionGCStatus operator & (ObMultiVersionGCStatus a, ObMultiVersionGCStatus b) +{ + return static_cast(static_cast(a) & static_cast(b)); +} + +class ObMultiVersionSnapshotInfo +{ +public: + ObMultiVersionSnapshotInfo(); + ObMultiVersionSnapshotInfo(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr); + share::SCN snapshot_version_; + ObMultiVersionSnapshotType snapshot_type_; + ObMultiVersionGCStatus status_; + int64_t create_time_; + ObAddr addr_; + + TO_STRING_KV(K_(snapshot_version), + K_(snapshot_type), + K_(status), + K_(create_time), + K_(addr)); +}; + +// Because of the value semantics of the ObFunction, we will also use the +// encapsulation of the class operator to implement the state machine. You can +// read the ObMultiVersionGCSnapshotOperator and its usage of lambda as an +// example. +using ObMultiVersionGCSnapshotFunction = + ObFunction; + +// Because of the value semantics of the ObFunction, we use class and operator +// to implement the state transfer of the state machine under the collect function +// of the ObMultiVersionGCSnapshotCollector. +class ObMultiVersionGCSnapshotFunctor +{ +public: + virtual int operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr) = 0; + + VIRTUAL_TO_STRING_KV("Operator", "MultiVersionGC"); +}; + +// ObMultiVersionGCSnapshotCalculator is used for tenant freezer manager to +// calculate the min reserved multi-version snapshot and cache some information. +class ObMultiVersionGCSnapshotCalculator : public ObMultiVersionGCSnapshotFunctor +{ +public: + ObMultiVersionGCSnapshotCalculator(); + ~ObMultiVersionGCSnapshotCalculator(); + int operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr); + share::SCN get_reserved_snapshot_version() const; + ObMultiVersionGCStatus get_status() const; + bool is_this_server_disabled() const + { return is_this_server_disabled_; } + + VIRTUAL_TO_STRING_KV(K_(reserved_snapshot_version), + K_(reserved_snapshot_type), + K_(reserved_status), + K_(reserved_create_time), + K_(reserved_addr), + K_(is_this_server_disabled), + K_(status)); +private: + share::SCN reserved_snapshot_version_; + ObMultiVersionSnapshotType reserved_snapshot_type_; + ObMultiVersionGCStatus reserved_status_; + int64_t reserved_create_time_; + ObAddr reserved_addr_; + // whether this server is disabled for gc status + bool is_this_server_disabled_; + // final status after transve all gc status + ObMultiVersionGCStatus status_; +}; + +class ObMultiVersionGCSnapshotCollector : public ObMultiVersionGCSnapshotFunctor +{ +public: + ObMultiVersionGCSnapshotCollector(ObIArray &array); + ~ObMultiVersionGCSnapshotCollector(); + int operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr); + + VIRTUAL_TO_STRING_KV(K_(snapshots_info)); +private: + ObIArray &snapshots_info_; +}; + +class ObMultiVersionGCSnapshotOperator : public ObMultiVersionGCSnapshotFunctor +{ +public: + ObMultiVersionGCSnapshotOperator(const ObMultiVersionGCSnapshotFunction &func); + int operator()(const share::SCN snapshot_version, + const ObMultiVersionSnapshotType snapshot_type, + const ObMultiVersionGCStatus status, + const int64_t create_time, + const ObAddr addr); +private: + ObMultiVersionGCSnapshotFunction func_; +}; + +class GetMinActiveSnapshotVersionFunctor +{ +public: + GetMinActiveSnapshotVersionFunctor() + : min_active_snapshot_version_(share::SCN::max_scn()) {} + virtual ~GetMinActiveSnapshotVersionFunctor() {} + bool operator()(sql::ObSQLSessionMgr::Key key, sql::ObSQLSessionInfo *sess_info); + share::SCN get_min_active_snapshot_version() + { return min_active_snapshot_version_; } +private: + share::SCN min_active_snapshot_version_; +}; + +// OceanBase 4.0 reclaims multi-version data through a globally incremented +// timestamp. It's the multi-version data that is less than the specified +// timestamp except for the latest version can be safely recycled. However, the +// setting of this timestamp is worth deliberating: If this timestamp is too +// small, the snapshot of long-running txns will be lost; if the timestamp is +// too large, a large amount of the multi-version data can not be recycled, +// affecting query efficiency and disk utilization. +// +// Therefore, in order to efficiently recover these multi-version data, the +// class ObMultiVersionGarbageCollector currenly detects the minimal active txn +// snapshot globally. And decides the recyclable timestamp using this minimum +// active txn snapshot. This method balances the difficulty of implementation +// and the benefits it brings +// +// In the future, we will expand to further implementations that bring more +// benefiets based on different competing products and papers, including but not +// limited to optimizing the reclaimation with only some range of the +// multi-version data; and optimizing based on table-level visibility, etc. +// +// Above all, we need to achieve the following 3 rules: +// 1. All data that will no longer be read needs to be recycled as soon as possible +// 2. All data that may be read must be preserved +// 3. Guarantee user-understandable recovery in abnormal scenarios +class ObMultiVersionGarbageCollector +{ +public: + ObMultiVersionGarbageCollector(); + ~ObMultiVersionGarbageCollector(); + static int mtl_init(ObMultiVersionGarbageCollector *&p_garbage_colloector); +public: + int init(); + int start(); + int stop(); + void wait(); + void destroy(); + // cure means treat myself for the injurity. It resets all state just like + // treat a patient + void cure(); + // repeat_xxx will repeatably work for its task under task or meet the time + // requirement. Currently we retry task every 1 minute and rework the task + // every 10 minute according to unchangable configuration. + // + // study means learn for the different ObMultiVersionSnapshotType and report + // to the inner table. refresh means refresh globally reserved snapshots. + // reclaim means reclaim long-time unchangable value in the inner table. + void repeat_study(); + void repeat_refresh(); + void repeat_reclaim(); + // collect will invoke the functor for every entries in the inner table. + // It is flexible to use. + // NB: it will stop if functor report error + int collect(ObMultiVersionGCSnapshotFunctor& functor); + // report will report the following four entries into the inner table. + // NB: the 4 entries must be inserted atomically for the rule2. + int report(const share::SCN min_unallocated_GTS, + const share::SCN min_unallocated_WRS, + const share::SCN max_committed_txn_version, + const share::SCN min_active_txn_version); + // update will update the four entries with the new status. + // NB: the 4 entries must be updated atomically for the rule2. + int update_status(const ObMultiVersionGCStatus status); + // reclaim will remove the four entries for expired nodes. + // NB: the 4 entries must be removed atomically for the rule2. + int reclaim(); + + // get_reserved_snapshot_for_active_txn fetch the cached globally reserved + // snapshot if updated in time, otherwise max_scn() is used for available + share::SCN get_reserved_snapshot_for_active_txn() const; + // is_gc_disabled shows the global gc status of whether the gc is disabled + bool is_gc_disabled() const; + + TO_STRING_KV(KP(this), + K_(last_study_timestamp), + K_(last_refresh_timestamp), + K_(last_reclaim_timestamp), + K_(has_error_when_study), + K_(refresh_error_too_long), + K_(has_error_when_reclaim), + K_(gc_is_disabled), + K_(global_reserved_snapshot), + K_(is_inited)); + +public: + static int64_t GARBAGE_COLLECT_RETRY_INTERVAL; + static int64_t GARBAGE_COLLECT_EXEC_INTERVAL; + static int64_t GARBAGE_COLLECT_PRECISION; + static int64_t GARBAGE_COLLECT_RECLAIM_DURATION; +private: + int study(); + int refresh_(); + int disk_monitor_(const bool is_this_server_disabled); + int monitor_(const ObArray &snapshot_servers); + int reclaim_(const ObArray &reclaimable_servers); + int study_min_unallocated_GTS(share::SCN &min_unallocated_GTS); + int study_min_unallocated_WRS(share::SCN &min_unallocated_WRS); + int study_max_committed_txn_version(share::SCN &max_committed_txn_version); + int study_min_active_txn_version(share::SCN &min_active_txn_version); + int is_disk_almost_full_(bool &is_almost_full); + void decide_gc_status_(const ObMultiVersionGCStatus gc_status); + void decide_reserved_snapshot_version_(const share::SCN reserved_snapshot); + + // ============== for test ================ + OB_NOINLINE bool can_report(); + OB_NOINLINE bool is_refresh_fail(); +private: + common::ObOccamTimer timer_; + common::ObOccamTimerTaskRAIIHandle timer_handle_; + + int64_t last_study_timestamp_; + int64_t last_refresh_timestamp_; + int64_t last_reclaim_timestamp_; + bool has_error_when_study_; + // refresh too long without contacting inner table successfully. + // It may be caused by inner table majority crash or network issues. + bool refresh_error_too_long_; + bool has_error_when_reclaim_; + // gc is disabled in inner table + // it may be disabled by disk mointor + bool gc_is_disabled_; + // globally reserved snapshot for active txn + share::SCN global_reserved_snapshot_; + bool is_inited_; +}; + +} // namespace concurrency_control +} // namespace oceanbase + +#endif // OCEANBASE_STORAGE_CONCURRENCY_CONTROL_OB_MULTI_VERSION_GARBAGE_COLLECTOR diff --git a/src/storage/ls/ob_freezer.cpp b/src/storage/ls/ob_freezer.cpp index a67d7c148..ffb67eb1e 100644 --- a/src/storage/ls/ob_freezer.cpp +++ b/src/storage/ls/ob_freezer.cpp @@ -437,6 +437,20 @@ int ObFreezer::tablet_freeze(const ObTabletID &tablet_id, ObFuture *result) { int ret = OB_SUCCESS; share::ObLSID ls_id = get_ls_id(); + + if (tablet_id.is_ls_inner_tablet()) { + if (OB_FAIL(freeze_ls_inner_tablet_(tablet_id))) { + TRANS_LOG(WARN, "freeze ls inner tablet failed", KR(ret), K(ls_id), K(tablet_id)); + } + } else if (OB_FAIL(freeze_normal_tablet_(tablet_id, result))) { + TRANS_LOG(WARN, "freeze normal tablet failed", KR(ret), K(ls_id), K(tablet_id)); + } + return ret; +} + +int ObFreezer::freeze_normal_tablet_(const ObTabletID &tablet_id, ObFuture *result) +{ + int ret = OB_SUCCESS; ObTabletHandle handle; ObTablet *tablet = nullptr; ObTabletMemtableMgr *memtable_mgr = nullptr; @@ -453,7 +467,6 @@ int ObFreezer::tablet_freeze(const ObTabletID &tablet_id, ObFuture *result) ret = OB_NOT_INIT; TRANS_LOG(WARN, "[Freezer] not inited", K(ret), K(ls_id), K(tablet_id)); } else if (OB_UNLIKELY(!enable_)) { - ret = OB_NOT_RUNNING; LOG_WARN("freezer is offline, can not freeze now", K(ret), K(ls_id)); } else if (OB_FAIL(guard.try_set_tablet_freeze_begin())) { // no need freeze now, a ls freeze is running or will be running @@ -510,6 +523,18 @@ int ObFreezer::tablet_freeze(const ObTabletID &tablet_id, ObFuture *result) return ret; } +int ObFreezer::freeze_ls_inner_tablet_(const ObTabletID &tablet_id) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(get_ls_tx_svr())) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(WARN, "[Freezer] ls_tx_svr is unexpected nullptr", KR(ret), K(get_ls_id()), K(tablet_id)); + } else if (OB_FAIL(get_ls_tx_svr()->flush_ls_inner_tablet(tablet_id))) { + TRANS_LOG(WARN, "[Freezer] freeze ls inner tablet failed", KR(ret), K(get_ls_id()), K(tablet_id)); + } + return ret; +} + int ObFreezer::force_tablet_freeze(const ObTabletID &tablet_id) { int ret = OB_SUCCESS; diff --git a/src/storage/ls/ob_freezer.h b/src/storage/ls/ob_freezer.h index 28d670fd3..ea4e6e16a 100644 --- a/src/storage/ls/ob_freezer.h +++ b/src/storage/ls/ob_freezer.h @@ -288,6 +288,8 @@ private: void set_ls_freeze_begin_(); void set_ls_freeze_end_(); int check_ls_state(); // must be used under the protection of ls_lock + int freeze_normal_tablet_(const ObTabletID &tablet_id, ObFuture *result = nullptr); + int freeze_ls_inner_tablet_(const ObTabletID &tablet_id); private: // flag whether the logsteram is freezing // the first bit: 1, freeze; 0, not freeze diff --git a/src/storage/ls/ob_ls_get_mod.h b/src/storage/ls/ob_ls_get_mod.h index 887781c41..3cc8ba116 100644 --- a/src/storage/ls/ob_ls_get_mod.h +++ b/src/storage/ls/ob_ls_get_mod.h @@ -38,7 +38,8 @@ enum class ObLSGetMod : int TXSTORAGE_MOD = 15, LEADER_COORDINATOR_MOD = 16, DATA_MEMTABLE_MOD = 17, - TOTAL_MAX_MOD = 18, + MULTI_VERSION_GARBAGE_COLLOECTOR_MOD = 18, + TOTAL_MAX_MOD = 19, }; } diff --git a/src/storage/ls/ob_ls_tablet_service.cpp b/src/storage/ls/ob_ls_tablet_service.cpp index da48d1ac5..85d6a2e3c 100644 --- a/src/storage/ls/ob_ls_tablet_service.cpp +++ b/src/storage/ls/ob_ls_tablet_service.cpp @@ -837,6 +837,7 @@ int ObLSTabletService::table_scan(ObTableScanIterator &iter, ObTableScanParam &p } } NG_TRACE(S_table_scan_end); + return ret; } @@ -2390,7 +2391,7 @@ int ObLSTabletService::insert_rows( } NG_TRACE(S_insert_rows_end); - return ret; + return ret; } int ObLSTabletService::insert_row( diff --git a/src/storage/ls/ob_ls_tx_service.cpp b/src/storage/ls/ob_ls_tx_service.cpp index 6f7852d6f..cf21775bb 100644 --- a/src/storage/ls/ob_ls_tx_service.cpp +++ b/src/storage/ls/ob_ls_tx_service.cpp @@ -154,6 +154,7 @@ int ObLSTxService::get_read_store_ctx(const SCN &snapshot, int ObLSTxService::get_write_store_ctx(ObTxDesc &tx, const ObTxReadSnapshot &snapshot, + const concurrent_control::ObWriteFlag write_flag, storage::ObStoreCtx &store_ctx) const { int ret = OB_SUCCESS; @@ -161,7 +162,7 @@ int ObLSTxService::get_write_store_ctx(ObTxDesc &tx, ret = OB_NOT_INIT; TRANS_LOG(WARN, "not init", K(ret)); } else { - ret = trans_service_->get_write_store_ctx(tx, snapshot, store_ctx); + ret = trans_service_->get_write_store_ctx(tx, snapshot, write_flag, store_ctx); } return ret; } @@ -569,6 +570,7 @@ int ObLSTxService::traversal_flush() return ret; } + void ObLSTxService::reset_() { WLockGuard guard(rwlock_); for (int i = 0; i < ObCommonCheckpointType::MAX_BASE_TYPE; i++) { diff --git a/src/storage/ls/ob_ls_tx_service.h b/src/storage/ls/ob_ls_tx_service.h index 459bec351..d2ab16597 100644 --- a/src/storage/ls/ob_ls_tx_service.h +++ b/src/storage/ls/ob_ls_tx_service.h @@ -94,6 +94,7 @@ public: ObStoreCtx &store_ctx) const; int get_write_store_ctx(transaction::ObTxDesc &tx, const transaction::ObTxReadSnapshot &snapshot, + const concurrent_control::ObWriteFlag write_flag, storage::ObStoreCtx &store_ctx) const; int revert_store_ctx(storage::ObStoreCtx &store_ctx) const; // Freeze process needs to traverse trans ctx to submit redo log diff --git a/src/storage/memtable/mvcc/ob_mvcc_acc_ctx.h b/src/storage/memtable/mvcc/ob_mvcc_acc_ctx.h index 8b2dac26d..36636bd3d 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_acc_ctx.h +++ b/src/storage/memtable/mvcc/ob_mvcc_acc_ctx.h @@ -17,6 +17,7 @@ #include "storage/tx/ob_trans_define.h" #include "lib/oblog/ob_log.h" #include "lib/oblog/ob_log_module.h" +#include "storage/memtable/ob_concurrent_control.h" namespace oceanbase { @@ -48,6 +49,7 @@ public: tx_ctx_(NULL), mem_ctx_(NULL), tx_scn_(-1), + write_flag_(), handle_start_time_(OB_INVALID_TIMESTAMP) {} ~ObMvccAccessCtx() { @@ -59,6 +61,7 @@ public: tx_ctx_ = NULL; mem_ctx_ = NULL; tx_scn_ = -1; + write_flag_.reset(); handle_start_time_ = OB_INVALID_TIMESTAMP; } void reset() { @@ -75,6 +78,7 @@ public: tx_ctx_ = NULL; mem_ctx_ = NULL; tx_scn_ = -1; + write_flag_.reset(); handle_start_time_ = OB_INVALID_TIMESTAMP; } bool is_valid() const { @@ -141,7 +145,8 @@ public: const storage::ObTxTableGuard &tx_table_guard, const transaction::ObTxSnapshot &snapshot, const int64_t abs_lock_timeout, - const int64_t tx_lock_timeout) + const int64_t tx_lock_timeout, + const concurrent_control::ObWriteFlag write_flag) { reset(); type_ = T::WRITE; @@ -154,6 +159,7 @@ public: snapshot_ = snapshot; abs_lock_timeout_ = abs_lock_timeout; tx_lock_timeout_ = tx_lock_timeout; + write_flag_ = write_flag; } void init_replay(transaction::ObPartTransCtx &tx_ctx, ObMemtableCtx &mem_ctx, @@ -212,7 +218,8 @@ public: KPC_(tx_desc), KP_(tx_ctx), KP_(mem_ctx), - K_(tx_scn)); + K_(tx_scn), + K_(write_flag)); private: void warn_tx_ctx_leaky_(); public: // NOTE: those field should only be accessed by txn relative routine @@ -235,10 +242,11 @@ public: // NOTE: those field should only be accessed by txn relative routine storage::ObTxTableGuard tx_table_guard_; // specials for MvccWrite transaction::ObTransID tx_id_; - transaction::ObTxDesc *tx_desc_; // the txn descriptor - transaction::ObPartTransCtx *tx_ctx_; // the txn context - ObMemtableCtx *mem_ctx_; // memtable-ctx - int64_t tx_scn_; // the change's number of this modify + transaction::ObTxDesc *tx_desc_; // the txn descriptor + transaction::ObPartTransCtx *tx_ctx_; // the txn context + ObMemtableCtx *mem_ctx_; // memtable-ctx + int64_t tx_scn_; // the change's number of this modify + concurrent_control::ObWriteFlag write_flag_; // the write flag of the write process // this was used for runtime mertic int64_t handle_start_time_; diff --git a/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp b/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp index 3950ff9d3..074288a8f 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_ctx.cpp @@ -272,12 +272,15 @@ ObMvccWriteGuard::~ObMvccWriteGuard() int ret = OB_SUCCESS; auto tx_ctx = ctx_->get_trans_ctx(); ctx_->write_done(); - if (OB_NOT_NULL(memtable_) && memtable_->is_frozen_memtable()) { - if (OB_FAIL(tx_ctx->submit_redo_log(true/*is_freeze*/))) { + if (OB_NOT_NULL(memtable_)) { + bool is_freeze = memtable_->is_frozen_memtable(); + if (OB_FAIL(tx_ctx->submit_redo_log(is_freeze))) { if (REACH_TIME_INTERVAL(100 * 1000)) { - TRANS_LOG(WARN, "failed to submit freeze log", K(ret), KPC(tx_ctx)); + TRANS_LOG(WARN, "failed to submit log if neccesary", K(ret), K(is_freeze), KPC(tx_ctx)); + } + if (is_freeze) { + memtable_->get_freezer()->set_need_resubmit_log(true); } - memtable_->get_freezer()->set_need_resubmit_log(true); } } } diff --git a/src/storage/memtable/mvcc/ob_mvcc_define.h b/src/storage/memtable/mvcc/ob_mvcc_define.h index c3e25d7b5..386a009e0 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_define.h +++ b/src/storage/memtable/mvcc/ob_mvcc_define.h @@ -103,7 +103,7 @@ struct ObMvccWriteResult { // row) bool need_insert_; // is_new_locked_ indicates whether you are locking the row for the first - // time(mainly used for detecting errors) + // time for your txn(mainly used for deadlock detector and detecting errors) bool is_new_locked_; // lock_state_ is used for deadlock detector and lock wait mgr storage::ObStoreRowLockState lock_state_; diff --git a/src/storage/memtable/mvcc/ob_mvcc_engine.cpp b/src/storage/memtable/mvcc/ob_mvcc_engine.cpp index 9410c646c..b283866dc 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_engine.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_engine.cpp @@ -307,7 +307,8 @@ int ObMvccEngine::create_kv( } int ObMvccEngine::mvcc_write(ObIMemtableCtx &ctx, - const SCN snapshot_version, + const concurrent_control::ObWriteFlag write_flag, + const transaction::ObTxSnapshot &snapshot, ObMvccRow &value, const ObTxNodeArg &arg, ObMvccWriteResult &res) @@ -318,7 +319,8 @@ int ObMvccEngine::mvcc_write(ObIMemtableCtx &ctx, if (OB_FAIL(build_tx_node_(ctx, arg, node))) { TRANS_LOG(WARN, "build tx node failed", K(ret), K(ctx), K(arg)); } else if (OB_FAIL(value.mvcc_write(ctx, - snapshot_version, + write_flag, + snapshot, *node, res))) { if (OB_TRY_LOCK_ROW_CONFLICT != ret && diff --git a/src/storage/memtable/mvcc/ob_mvcc_engine.h b/src/storage/memtable/mvcc/ob_mvcc_engine.h index 3813247dc..2f5557b82 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_engine.h +++ b/src/storage/memtable/mvcc/ob_mvcc_engine.h @@ -68,7 +68,8 @@ public: // OB_TRANSACTION_SET_VIOLATION if encountering lost update. The interesting // implementation about mvcc_write is located in ob_mvcc_row.cpp/.h int mvcc_write(ObIMemtableCtx &ctx, - const share::SCN snapshot_version, + const concurrent_control::ObWriteFlag write_flag, + const transaction::ObTxSnapshot &snapshot, ObMvccRow &value, const ObTxNodeArg &arg, ObMvccWriteResult &res); diff --git a/src/storage/memtable/mvcc/ob_mvcc_row.cpp b/src/storage/memtable/mvcc/ob_mvcc_row.cpp index b1636e7ea..d65a66c91 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_row.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_row.cpp @@ -25,6 +25,7 @@ #include "storage/memtable/ob_lock_wait_mgr.h" #include "storage/tx/ob_trans_part_ctx.h" #include "storage/memtable/ob_memtable_context.h" +#include "storage/memtable/ob_concurrent_control.h" #include "storage/tx/ob_trans_ctx.h" #include "storage/tx/ob_trans_event.h" #include "storage/memtable/mvcc/ob_mvcc_trans_ctx.h" @@ -369,8 +370,8 @@ int64_t ObMvccRow::to_string(char *buf, const int64_t buf_len) const to_cstring(max_elr_trans_version_), max_elr_trans_id_.get_id(), latest_compact_ts_, - total_trans_node_cnt_, last_compact_cnt_, + total_trans_node_cnt_, max_modify_count_, min_modify_count_); return pos; @@ -878,8 +879,9 @@ int ObMvccRow::wakeup_waiter(const ObTabletID &tablet_id, } int ObMvccRow::mvcc_write_(ObIMemtableCtx &ctx, + const concurrent_control::ObWriteFlag write_flag, ObMvccTransNode &writer_node, - const SCN snapshot_version, + const transaction::ObTxSnapshot &snapshot, ObMvccWriteResult &res) { int ret = OB_SUCCESS; @@ -889,7 +891,9 @@ int ObMvccRow::mvcc_write_(ObIMemtableCtx &ctx, ObTxTableGuard *tx_table_guard = ctx.get_tx_table_guard(); ObTxTable *tx_table = tx_table_guard->get_tx_table(); int64_t read_epoch = tx_table_guard->epoch(); - ObTransID write_tx_id = ctx.get_tx_id(); + ObTransID writer_tx_id = ctx.get_tx_id(); + const SCN snapshot_version = snapshot.version_; + const int64_t reader_seq_no = snapshot.scn_; bool &can_insert = res.can_insert_; bool &need_insert = res.need_insert_; bool &is_new_locked = res.is_new_locked_; @@ -938,7 +942,7 @@ int ObMvccRow::mvcc_write_(ObIMemtableCtx &ctx, // so we need look for the next one iter = iter->prev_; need_retry = true; - } else if (data_tx_id == write_tx_id) { + } else if (data_tx_id == writer_tx_id) { // Case 4: the newest node is not decided and locked by itself, so we // can insert into it bool is_lock_node = false; @@ -968,6 +972,7 @@ int ObMvccRow::mvcc_write_(ObIMemtableCtx &ctx, lock_state.is_locked_ = true; lock_state.lock_trans_id_ = data_tx_id; lock_state.lock_data_sequence_ = iter->get_seq_no(); + lock_state.lock_dml_flag_ = iter->get_dml_flag(); lock_state.is_delayed_cleanout_ = iter->is_delayed_cleanout(); lock_state.mvcc_row_ = this; } @@ -976,9 +981,19 @@ int ObMvccRow::mvcc_write_(ObIMemtableCtx &ctx, if (OB_SUCC(ret)) { if (can_insert && need_insert) { - if (OB_SUCC(check_double_insert_(snapshot_version, - writer_node, - list_head_))) { + if (nullptr != list_head_ && + OB_FAIL(concurrent_control::check_sequence_set_violation(write_flag, + reader_seq_no, + writer_tx_id, + writer_node.get_dml_flag(), + writer_node.get_seq_no(), + list_head_->get_tx_id(), + list_head_->get_dml_flag(), + list_head_->get_seq_no()))) { + TRANS_LOG(WARN, "check sequence set violation failed", K(ret), KPC(this)); + } else if (OB_SUCC(check_double_insert_(snapshot_version, + writer_node, + list_head_))) { ATOMIC_STORE(&(writer_node.prev_), list_head_); ATOMIC_STORE(&(writer_node.next_), NULL); if (NULL != list_head_) { @@ -1045,11 +1060,13 @@ void ObMvccRow::mvcc_undo() } int ObMvccRow::mvcc_write(ObIMemtableCtx &ctx, - const SCN snapshot_version, + const concurrent_control::ObWriteFlag write_flag, + const transaction::ObTxSnapshot &snapshot, ObMvccTransNode &node, ObMvccWriteResult &res) { int ret = OB_SUCCESS; + const SCN snapshot_version = snapshot.version_; lock_begin(ctx); if (max_trans_version_.atomic_load() > snapshot_version @@ -1059,7 +1076,11 @@ int ObMvccRow::mvcc_write(ObIMemtableCtx &ctx, TRANS_LOG(WARN, "transaction set violation", K(ret), K(snapshot_version), "txNode_to_write", node, "memtableCtx", ctx, "mvccRow", PC(this)); - } else if (OB_FAIL(mvcc_write_(ctx, node, snapshot_version, res))) { + } else if (OB_FAIL(mvcc_write_(ctx, + write_flag, + node, + snapshot, + res))) { TRANS_LOG(WARN, "mvcc write failed", K(ret), K(node), K(ctx)); } else if (!res.can_insert_) { // Case1: Cannot insert because of write-write conflict @@ -1111,7 +1132,7 @@ int ObMvccRow::check_row_locked(ObMvccAccessCtx &ctx, ObStoreRowLockState &lock_ if (OB_ISNULL(iter)) { // Case 1: head is empty, so node currently is not be locked lock_state.is_locked_ = false; - lock_state.trans_version_ = get_max_trans_version(); + lock_state.trans_version_.set_min(); lock_state.lock_trans_id_.reset(); need_retry = false; } else { @@ -1135,10 +1156,11 @@ int ObMvccRow::check_row_locked(ObMvccAccessCtx &ctx, ObStoreRowLockState &lock_ need_retry = true; } else { lock_state.is_locked_ = true; - lock_state.trans_version_ = get_max_trans_version(); + lock_state.trans_version_.set_min(); lock_state.lock_trans_id_= data_tx_id; - lock_state.is_delayed_cleanout_ = iter->is_delayed_cleanout(); lock_state.lock_data_sequence_ = iter->get_seq_no(); + lock_state.lock_dml_flag_ = iter->get_dml_flag(); + lock_state.is_delayed_cleanout_ = iter->is_delayed_cleanout(); need_retry = false; } } diff --git a/src/storage/memtable/mvcc/ob_mvcc_row.h b/src/storage/memtable/mvcc/ob_mvcc_row.h index e2274884f..d5f68ebc0 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_row.h +++ b/src/storage/memtable/mvcc/ob_mvcc_row.h @@ -215,7 +215,8 @@ struct ObMvccRow // is_new_locked returns whether node represents the first lock for the operation // conflict_tx_id if write failed this field indicate the txn-id which hold the lock of current row int mvcc_write(ObIMemtableCtx &ctx, - const share::SCN snapshot_version, + const concurrent_control::ObWriteFlag write_flag, + const transaction::ObTxSnapshot &snapshot, ObMvccTransNode &node, ObMvccWriteResult &res); @@ -332,8 +333,9 @@ struct ObMvccRow // ===================== ObMvccRow Private Function ===================== int mvcc_write_(ObIMemtableCtx &ctx, + const concurrent_control::ObWriteFlag write_flag, ObMvccTransNode &node, - const share::SCN snapshot_version, + const transaction::ObTxSnapshot &snapshot, ObMvccWriteResult &res); // ===================== ObMvccRow Protection Code ===================== diff --git a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp index 360aacc6c..5a37d70fb 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp @@ -708,6 +708,12 @@ int ObMvccRowCallback::del() 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 + transaction::ObPartTransCtx *part_ctx = static_cast(get_trans_ctx()); + part_ctx->clear_block_frozen_memtable(); + ret = remove(); return ret; } diff --git a/src/storage/memtable/mvcc/ob_query_engine.cpp b/src/storage/memtable/mvcc/ob_query_engine.cpp index ae33a9434..aba572c7a 100644 --- a/src/storage/memtable/mvcc/ob_query_engine.cpp +++ b/src/storage/memtable/mvcc/ob_query_engine.cpp @@ -50,6 +50,43 @@ void ObQueryEngine::TableIndex::destroy() keybtree_.destroy(); } +void ObQueryEngine::TableIndex::check_cleanout(bool &is_all_cleanout, + bool &is_all_delay_cleanout, + int64_t &count) +{ + int ret = OB_SUCCESS; + Iterator iter; + ObStoreRowkeyWrapper scan_start_key_wrapper(&ObStoreRowkey::MIN_STORE_ROWKEY); + ObStoreRowkeyWrapper scan_end_key_wrapper(&ObStoreRowkey::MAX_STORE_ROWKEY); + iter.reset(); + const_cast(iter.get_key())->encode(nullptr); + if (IS_NOT_INIT) { + TRANS_LOG(WARN, "not init", "this", this); + } else if (OB_FAIL(keybtree_.set_key_range(iter.get_read_handle(), + scan_start_key_wrapper, 1, + scan_end_key_wrapper, 1, INT64_MAX))) { + TRANS_LOG(ERROR, "set key range to btree scan handle fail", KR(ret)); + } else { + blocksstable::ObRowReader row_reader; + blocksstable::ObDatumRow datum_row; + is_all_cleanout = true; + is_all_delay_cleanout = true; + count = 0; + for (int64_t row_idx = 0; OB_SUCC(ret) && OB_SUCC(iter.next_internal(true)); row_idx++) { + const ObMemtableKey *key = iter.get_key(); + ObMvccRow *row = iter.get_value(); + for (ObMvccTransNode *node = row->get_list_head(); OB_SUCC(ret) && OB_NOT_NULL(node); node = node->prev_) { + if (node->is_delayed_cleanout()) { + is_all_cleanout = false; + } else { + is_all_delay_cleanout = false; + } + count++; + } + } + } +} + void ObQueryEngine::TableIndex::dump2text(FILE* fd) { int ret = OB_SUCCESS; @@ -702,6 +739,17 @@ int ObQueryEngine::estimate_row_count(const ObMemtableKey *start_key, const int return ret; } +void ObQueryEngine::check_cleanout(bool &is_all_cleanout, + bool &is_all_delay_cleanout, + int64_t &count) +{ + if (OB_NOT_NULL(index_)) { + index_->check_cleanout(is_all_cleanout, + is_all_delay_cleanout, + count); + } +} + void ObQueryEngine::dump2text(FILE *fd) { TableIndex *index = ATOMIC_LOAD(&index_); diff --git a/src/storage/memtable/mvcc/ob_query_engine.h b/src/storage/memtable/mvcc/ob_query_engine.h index 9b325a8cd..4341896b0 100644 --- a/src/storage/memtable/mvcc/ob_query_engine.h +++ b/src/storage/memtable/mvcc/ob_query_engine.h @@ -151,6 +151,9 @@ public: ~TableIndex() { destroy(); } int init(); void destroy(); + void check_cleanout(bool &is_all_cleanout, + bool &is_all_delay_cleanout, + int64_t &count); void dump2text(FILE* fd); int dump_keyhash(FILE *fd) const; int dump_keybtree(FILE *fd); @@ -232,6 +235,9 @@ public: ? index->btree_alloc_memory() + btree_allocator_.get_allocated() : 0; } + void check_cleanout(bool &is_all_cleanout, + bool &is_all_delay_cleanout, + int64_t &count); void dump2text(FILE *fd); int get_table_index(TableIndex *&return_ptr) const; int set_table_index(const int64_t obj_cnt, TableIndex *&return_ptr); diff --git a/src/storage/memtable/ob_concurrent_control.cpp b/src/storage/memtable/ob_concurrent_control.cpp new file mode 100644 index 000000000..88f1bee18 --- /dev/null +++ b/src/storage/memtable/ob_concurrent_control.cpp @@ -0,0 +1,121 @@ +/** + * 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/memtable/ob_concurrent_control.h" + +namespace oceanbase +{ +namespace concurrent_control +{ + +OB_SERIALIZE_MEMBER(ObWriteFlag, flag_); + +// Currently, we consider the ability to implement the serial read and write +// capabilities for the single-row using the sequence number. Sequence number +// guarantees the following three principles. +// 1. The read sequence number is equal between all task in this statement. +// 2. The write sequence number is greater than the read sequence number of this +// statement. +// 3. The reader seq no is bigger or equal than the seq no of the last +// statement. +// +// With the above principles, we can prevent the concurrent operations of +// writing to the same data row in the same statement. That is, during writing +// to a mvcc row, we will check whether there is a data with the same txn id +// providing the lock semantic. And if its write sequence number is greater than +// or equal to the read sequence number of the current write operation, there +// may be a conflict. +// +// NB: Currently, there may be some cases that the txn sequentially increases +// the sequence number of the same row in the same task. And after paying great +// effort to sort out these posibilities(listed in the following cases), new +// possibility should be added carefully and remind the owner of the code. +int check_sequence_set_violation(const concurrent_control::ObWriteFlag write_flag, + const int64_t reader_seq_no, + const transaction::ObTransID writer_tx_id, + const blocksstable::ObDmlFlag writer_dml_flag, + const int64_t writer_seq_no, + const transaction::ObTransID locker_tx_id, + const blocksstable::ObDmlFlag locker_dml_flag, + const int64_t locker_seq_no) +{ + int ret = OB_SUCCESS; + // TODO(handora.qc): add flag to carefully screen out the different cases. For + // example, add the lob flag for the case 2.1 to prevent the other scenes from + // happening. + + if (writer_tx_id == locker_tx_id + // For statements during sql and threads during PDML, the following rules is + // guaranteed: + // 1. reader seq no is bigger or equal than the seq no of the last statements + && reader_seq_no < locker_seq_no) { + // Case 1: It may happens that two pdml unique index tasks insert the same + // row concurrently, so we report duplicate key under the case to prevent + // the insertion. + if (blocksstable::ObDmlFlag::DF_INSERT == writer_dml_flag + && blocksstable::ObDmlFlag::DF_INSERT == locker_dml_flag) { + ret = OB_ERR_PRIMARY_KEY_DUPLICATE; + TRANS_LOG(WARN, "pdml duplicate primary key found", K(ret), + K(writer_tx_id), K(writer_dml_flag), K(writer_seq_no), + K(locker_tx_id), K(locker_dml_flag), K(locker_seq_no)); + // Case 2.1: For the case of the update in the storage layer, it may be + // split into lock and update in a single statement and fail the check, so + // we need bypass this case(Currently only the update of the lob will cause + // it). We use the common idea that all operations split in the storage + // layer will use same sequence number, so we bypass the check if the writer + // sequence number is equal to the locker sequence number. + // } else if (writer_seq_no == locker_seq_no && + // (blocksstable::ObDmlFlag::DF_UPDATE == writer_dml_flag + // && blocksstable::ObDmlFlag::DF_LOCK == locker_dml_flag)) { + // + // Case 2.2: For the case of the self reference of the foreign key, it may + // be split into lock and insert/update in a single statement, so we need + // bypass this case(TODO(handora.qc): remove the requirement after yichang's + // modification). + } else if (blocksstable::ObDmlFlag::DF_LOCK == locker_dml_flag) { + // bypass the case + // Case 3: For the case of the update of the primary key in the sql layer, + // it will be split into a delete of the old rowkey and an insert for the + // new one which will fail to pass the check. And the two operations may + // even be split into two sequentially ordered steps to follow the sql-layer + // semantics, so we need bypass this case. + } else if (blocksstable::ObDmlFlag::DF_INSERT == writer_dml_flag + && blocksstable::ObDmlFlag::DF_DELETE == locker_dml_flag) { + // bypass the case + // Case 4: For the case of the insert of two same rowkeys with insert onto + // duplicate, it will be split into a insert of the rowkey and an update or + // delete of the same one which will fail to pass the check. And the two + // operations is split into two sequentially ordered steps so we need bypass + // this case. + } else if ((blocksstable::ObDmlFlag::DF_UPDATE == writer_dml_flag + || blocksstable::ObDmlFlag::DF_DELETE == writer_dml_flag) + && blocksstable::ObDmlFlag::DF_INSERT == locker_dml_flag) { + // bypass the case + // Case 5: For the case of table api, it inserts rows under the same stmt, + // and so fail to pass the check. We must bypass the case. + } else if (write_flag.is_table_api()) { + // bypass the case + } else { + // Case 6: It will never happen that two operaions on the same row for the + // same txn except the above cases. So we should report unexpected error. + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(ERROR, "multiple modification on one row found", K(reader_seq_no), + K(writer_tx_id), K(writer_dml_flag), K(writer_seq_no), + K(locker_tx_id), K(locker_dml_flag), K(locker_seq_no)); + } + } + + return ret; +} + +} // namespace concurrent_control +} // namespace oceanbase diff --git a/src/storage/memtable/ob_concurrent_control.h b/src/storage/memtable/ob_concurrent_control.h new file mode 100644 index 000000000..d99363add --- /dev/null +++ b/src/storage/memtable/ob_concurrent_control.h @@ -0,0 +1,115 @@ +/** + * 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_STORAGE_MEMTABLE_OB_CONCURRENT_CONTROL +#define OCEANBASE_STORAGE_MEMTABLE_OB_CONCURRENT_CONTROL + +#include "storage/tx/ob_trans_define.h" +#include "storage/blocksstable/ob_datum_row.h" + +namespace oceanbase +{ +namespace concurrent_control +{ + +// write flag is used for write flag modification +struct ObWriteFlag +{ + #define OBWF_BIT_TABLE_API 1 + #define OBWF_BIT_TABLE_LOCK 1 + #define OBWF_BIT_MDS 1 + #define OBWF_BIT_RESERVED 61 + + static const uint64_t OBWF_MASK_TABLE_API = (0x1UL << OBWF_BIT_TABLE_API) - 1; + static const uint64_t OBWF_MASK_TABLE_LOCK = (0x1UL << OBWF_BIT_TABLE_LOCK) - 1; + static const uint64_t OBWF_MASK_MDS = (0x1UL << OBWF_BIT_MDS) - 1; + + union + { + uint64_t flag_; + struct + { + uint64_t is_table_api_ : OBWF_BIT_TABLE_API; // 0: false(default), 1: true + uint64_t is_table_lock_ : OBWF_BIT_TABLE_LOCK; // 0: false(default), 1: true + uint64_t is_mds_ : OBWF_BIT_MDS; // 0: false(default), 1: true + uint64_t reserved_ : OBWF_BIT_RESERVED; + }; + }; + + ObWriteFlag() : flag_(0) {} + void reset() { flag_ = 0; } + inline bool is_table_api() const { return is_table_api_; } + inline void set_is_table_api() { is_table_api_ = true; } + inline bool is_table_lock() const { return is_table_lock_; } + inline void set_is_table_lock() { is_table_lock_ = true; } + inline bool is_mds() const { return is_mds_; } + inline void set_is_mds() { is_mds_ = true; } + + TO_STRING_KV("is_table_api", is_table_api_, + "is_table_lock", is_table_lock_, + "is_mds", is_mds_); + + OB_UNIS_VERSION(1); +}; + +// TODO(handora.qc): Move more concurrent control related method into the function + +// In Oceanbase 4.0, in order to prevent concurrent unique key insertion txns +// from inserting the same row at the same time, we decompose the unique key +// insertion into three actions: 1. existence verification, 2. data insertion +// and 3. lost update detection. That is, based on the snapshot isolation +// concurrency Control, the inherently guaranteed serial read and write +// capabilities on a single row of data solves the problem of concurrent unique +// key insertion. For two txns, T1 and T2, if T1's commit version is lower than +// T2's read version, T2 will see T1's insertion through existence verification; +// If T1's commit version is bigger than T2's read version, T2 will not see it +// under snapshot, while T2 will report TSC when found T1's update. +// +// After supporting the PDML, the txn has increased the ability to operate data +// concurrently in the same statement. Especially in the case of an index, it +// will introduce the situation of concurrently operating the same row. +// Therefore, the mentioned method cannot solve the concurrent unique key +// insertion of the same txn in this case(Because TSC will not help for the same +// txn). +// +// We consider that the essence of the problem is that tasks do not support +// serial read and write capabilities for single-row data. Therefore, under this +// idea, we consider whether it is possible to transplant concurrency control +// for serial read and write capabilities of single-row. Therefore, we consider +// To use the sequence number. Sequence number guarantees the following three +// principles. +// 1. The read sequence number is equal between all task in this statement. +// 2. The write sequence number is greater than the read sequence number of this +// statement. +// 3. The reader seq no is bigger or equal than the seq no of the last +// statement. +// +// With the above guarantees, we can realize whether there are concurrent +// operations of reading and writing the same data row in the same statement. +// That is, we will Find whether there is a write operation of the same +// transaction, whose write sequence number is greater or equal than the read +// sequence number of this operation. If there is a conflict, this task will be +// rolled back. This solves the common problem between B and C mentioned above +// It is guaranteed that only There is a concurrent modification of a task. +int check_sequence_set_violation(const concurrent_control::ObWriteFlag write_flag, + const int64_t reader_seq_no, + const transaction::ObTransID checker_tx_id, + const blocksstable::ObDmlFlag checker_dml_flag, + const int64_t checker_seq_no, + const transaction::ObTransID locker_tx_id, + const blocksstable::ObDmlFlag locker_dml_flag, + const int64_t locker_seq_no); + +} // namespace concurrent_control +} // namespace oceanbase + +#endif // OCEANBASE_STORAGE_MEMTABLE_OB_CONCURRENT_CONTROL diff --git a/src/storage/memtable/ob_memtable.cpp b/src/storage/memtable/ob_memtable.cpp index 69ac1e782..613f8e85b 100644 --- a/src/storage/memtable/ob_memtable.cpp +++ b/src/storage/memtable/ob_memtable.cpp @@ -28,6 +28,7 @@ #include "storage/memtable/ob_memtable_util.h" #include "storage/memtable/ob_memtable_context.h" #include "storage/memtable/ob_lock_wait_mgr.h" +#include "storage/memtable/ob_concurrent_control.h" #include "storage/compaction/ob_tablet_merge_task.h" #include "storage/compaction/ob_schedule_dag_func.h" #include "storage/compaction/ob_compaction_diagnose.h" @@ -1123,12 +1124,16 @@ int ObMemtable::replay_row(ObStoreCtx &ctx, //////////////////////////////////////////////////////////////////////////////////////////////////// int ObMemtable::lock_row_on_frozen_stores_(ObStoreCtx &ctx, + const ObTxNodeArg &arg, const ObMemtableKey *key, ObMvccRow *value, const storage::ObTableReadInfo &read_info, - ObStoreRowLockState &lock_state) + ObMvccWriteResult &res) { int ret = OB_SUCCESS; + const int64_t reader_seq_no = ctx.mvcc_acc_ctx_.snapshot_.scn_; + ObStoreRowLockState &lock_state = res.lock_state_; + if (OB_ISNULL(value) || !ctx.mvcc_acc_ctx_.is_write() || NULL == key) { TRANS_LOG(WARN, "invalid param", KP(value), K(ctx), KP(key)); ret = OB_INVALID_ARGUMENT; @@ -1142,7 +1147,8 @@ int ObMemtable::lock_row_on_frozen_stores_(ObStoreCtx &ctx, const ObIArray *stores = nullptr; common::ObSEArray iter_tables; ctx.table_iter_->resume(); - auto my_tx_id = ctx.mvcc_acc_ctx_.get_tx_id(); + ObTransID my_tx_id = ctx.mvcc_acc_ctx_.get_tx_id(); + while (OB_SUCC(ret)) { ObITable *table_ptr = nullptr; if (OB_FAIL(ctx.table_iter_->get_next(table_ptr))) { @@ -1160,9 +1166,14 @@ int ObMemtable::lock_row_on_frozen_stores_(ObStoreCtx &ctx, ret = OB_SUCCESS; } if (OB_SUCC(ret)) { + // lock_is_decided means we have found either the lock that is locked by + // an active txn or the latest unlocked txn data. And after we have found + // either of the above situations, whether the row is locked is determined. + bool lock_is_decided = false; + stores = &iter_tables; // ignore active memtable - for (int64_t i = stores->count() - 2; OB_SUCC(ret) && i >= 0; i--) { + for (int64_t i = stores->count() - 2; OB_SUCC(ret) && !lock_is_decided && i >= 0; i--) { lock_state.reset(); if (NULL == stores->at(i)) { ret = OB_ERR_UNEXPECTED; @@ -1189,6 +1200,9 @@ int ObMemtable::lock_row_on_frozen_stores_(ObStoreCtx &ctx, } if (OB_SUCC(ret)) { row_locked |= lock_state.is_locked_; + lock_is_decided = + (lock_state.is_locked_) || // row is locked by an active txn(may be other or yourself) + (!lock_state.trans_version_.is_min()); // row is committed(the row is indead unlocked) if (lock_state.is_locked_ && my_tx_id != lock_state.lock_trans_id_) { ret = OB_TRY_LOCK_ROW_CONFLICT; } else if (max_trans_version < lock_state.trans_version_) { @@ -1214,6 +1228,19 @@ int ObMemtable::lock_row_on_frozen_stores_(ObStoreCtx &ctx, value->set_lower_lock_scaned(); TRANS_LOG(DEBUG, "lower lock check finish", K(*value), K(*stores)); + } else { + // there is the lock on frozen stores. + if (res.has_insert() && // we only need check when the node is exactly inserted + OB_FAIL(concurrent_control::check_sequence_set_violation(ctx.mvcc_acc_ctx_.write_flag_, + reader_seq_no, + my_tx_id, + arg.data_->dml_flag_, + arg.seq_no_, + lock_state.lock_trans_id_, + lock_state.lock_dml_flag_, + lock_state.lock_data_sequence_))) { + TRANS_LOG(WARN, "check sequence set violation failed", K(ret), KPC(this)); + } } } } @@ -2086,6 +2113,21 @@ int ObMemtable::print_stat() const return ret; } +int ObMemtable::check_cleanout(bool &is_all_cleanout, + bool &is_all_delay_cleanout, + int64_t &count) +{ + int ret = OB_SUCCESS; + + TRANS_LOG(INFO, "check_cleanout", K_(key)); + + query_engine_.check_cleanout(is_all_cleanout, + is_all_delay_cleanout, + count); + + return ret; +} + int ObMemtable::dump2text(const char *fname) { int ret = OB_SUCCESS; @@ -2381,14 +2423,14 @@ int ObMemtable::set_(ObStoreCtx &ctx, } else { ObMemtableData mtd(new_row.flag_.get_dml_flag(), len, buf); ObTxNodeArg arg(&mtd, /*memtable_data*/ - NULL == old_row ? NULL : &old_row_data, - timestamp_, /*memstore_version*/ - ctx.mvcc_acc_ctx_.tx_scn_ /*seq_no*/); + NULL == old_row ? NULL : &old_row_data, + timestamp_, /*memstore_version*/ + ctx.mvcc_acc_ctx_.tx_scn_ /*seq_no*/); if (OB_FAIL(mvcc_write_(ctx, - &mtk, - read_info, - arg, - is_new_locked))) { + &mtk, + read_info, + arg, + is_new_locked))) { if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) { TRANS_LOG(WARN, "mvcc write fail", K(mtk), K(ret)); @@ -2494,6 +2536,7 @@ int ObMemtable::mvcc_write_(storage::ObStoreCtx &ctx, ObMvccWriteResult res; ObIMemtableCtx *mem_ctx = ctx.mvcc_acc_ctx_.get_mem_ctx(); SCN snapshot_version = ctx.mvcc_acc_ctx_.get_snapshot_version(); + transaction::ObTxSnapshot &snapshot = ctx.mvcc_acc_ctx_.snapshot_; if (OB_FAIL(mvcc_engine_.create_kv(key, &stored_key, @@ -2502,7 +2545,8 @@ int ObMemtable::mvcc_write_(storage::ObStoreCtx &ctx, is_new_add))) { TRANS_LOG(WARN, "create kv failed", K(ret), K(arg), K(*key), K(ctx)); } else if (OB_FAIL(mvcc_engine_.mvcc_write(*mem_ctx, - snapshot_version, + ctx.mvcc_acc_ctx_.write_flag_, + snapshot, *value, arg, res))) { @@ -2521,10 +2565,11 @@ int ObMemtable::mvcc_write_(storage::ObStoreCtx &ctx, TRANS_LOG(WARN, "mvcc write fail", K(ret)); } } else if (OB_FAIL(lock_row_on_frozen_stores_(ctx, + arg, key, value, read_info, - res.lock_state_))) { + res))) { if (OB_UNLIKELY(!res.is_new_locked_) && OB_TRY_LOCK_ROW_CONFLICT == ret) { TRANS_LOG(ERROR, "double lock detected", K(*key), K(*value), K(ctx)); } diff --git a/src/storage/memtable/ob_memtable.h b/src/storage/memtable/ob_memtable.h index baa8b91ef..794e025df 100644 --- a/src/storage/memtable/ob_memtable.h +++ b/src/storage/memtable/ob_memtable.h @@ -461,6 +461,9 @@ public: // Print stat data in log. // For memtable debug. int print_stat() const; + int check_cleanout(bool &is_all_cleanout, + bool &is_all_delay_cleanout, + int64_t &count); int dump2text(const char *fname); INHERIT_TO_STRING_KV("ObITable", ObITable, KP(this), K_(timestamp), K_(state), K_(freeze_clock), K_(max_schema_version), K_(write_ref_cnt), K_(local_allocator), @@ -482,10 +485,11 @@ private: int lock_row_on_frozen_stores_( storage::ObStoreCtx &ctx, + const ObTxNodeArg &arg, const ObMemtableKey *key, ObMvccRow *value, const storage::ObTableReadInfo &read_info, - storage::ObStoreRowLockState &lock_state); + ObMvccWriteResult &res); int remove_unused_callback_for_uncommited_txn_(); diff --git a/src/storage/memtable/ob_redo_log_generator.cpp b/src/storage/memtable/ob_redo_log_generator.cpp index ff3d75832..ad464021b 100644 --- a/src/storage/memtable/ob_redo_log_generator.cpp +++ b/src/storage/memtable/ob_redo_log_generator.cpp @@ -93,6 +93,12 @@ int ObRedoLogGenerator::fill_redo_log(char *buf, 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())); + } } else { bool fake_fill = false; if (MutatorType::MUTATOR_ROW == iter->get_mutator_type()) { diff --git a/src/storage/ob_i_store.cpp b/src/storage/ob_i_store.cpp index 1255116b8..7dfc4483f 100644 --- a/src/storage/ob_i_store.cpp +++ b/src/storage/ob_i_store.cpp @@ -134,6 +134,7 @@ void ObStoreRowLockState::reset() trans_version_ = SCN::min_scn(); lock_trans_id_.reset(); lock_data_sequence_ = 0; + lock_dml_flag_ = blocksstable::ObDmlFlag::DF_NOT_EXIST; is_delayed_cleanout_ = false; mvcc_row_ = NULL; } diff --git a/src/storage/ob_i_store.h b/src/storage/ob_i_store.h index eecf379f3..4d54331e4 100644 --- a/src/storage/ob_i_store.h +++ b/src/storage/ob_i_store.h @@ -224,19 +224,27 @@ struct ObStoreRowLockState { public: ObStoreRowLockState() - : is_locked_(false), trans_version_(share::SCN::min_scn()), lock_trans_id_(), - lock_data_sequence_(0), is_delayed_cleanout_(false), - mvcc_row_(NULL) - {} + : is_locked_(false), + trans_version_(share::SCN::min_scn()), + lock_trans_id_(), + lock_data_sequence_(0), + lock_dml_flag_(blocksstable::ObDmlFlag::DF_NOT_EXIST), + is_delayed_cleanout_(false), + mvcc_row_(NULL) {} void reset(); - TO_STRING_KV(K_(is_locked), K_(trans_version), K_(lock_trans_id), - K_(lock_data_sequence), K_(is_delayed_cleanout), + TO_STRING_KV(K_(is_locked), + K_(trans_version), + K_(lock_trans_id), + K_(lock_data_sequence), + K_(lock_dml_flag), + K_(is_delayed_cleanout), KP_(mvcc_row)); bool is_locked_; share::SCN trans_version_; transaction::ObTransID lock_trans_id_; int64_t lock_data_sequence_; + blocksstable::ObDmlFlag lock_dml_flag_; bool is_delayed_cleanout_; memtable::ObMvccRow *mvcc_row_; }; diff --git a/src/storage/tablet/ob_tablet.cpp b/src/storage/tablet/ob_tablet.cpp index 62a4a659c..69a2e2e7b 100644 --- a/src/storage/tablet/ob_tablet.cpp +++ b/src/storage/tablet/ob_tablet.cpp @@ -1380,14 +1380,6 @@ int ObTablet::update_row( } } - 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, "submit log if neccesary failed", K(tmp_ret), K(store_ctx), - K(relative_table)); - } - } - return ret; } @@ -1431,14 +1423,6 @@ int ObTablet::insert_row_without_rowkey_check( } } - 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, "submit log if necessary failed", K(tmp_ret), K(store_ctx), - K(relative_table)); - } - } - return ret; } diff --git a/src/storage/tx/ob_ctx_tx_data.cpp b/src/storage/tx/ob_ctx_tx_data.cpp index 1af69791f..53e4889ff 100644 --- a/src/storage/tx/ob_ctx_tx_data.cpp +++ b/src/storage/tx/ob_ctx_tx_data.cpp @@ -39,7 +39,6 @@ namespace transaction int ObCtxTxData::init(ObLSTxCtxMgr *ctx_mgr, int64_t tx_id) { int ret = OB_SUCCESS; - ObTxTable *tx_table = nullptr; if (OB_ISNULL(ctx_mgr) || tx_id < 0) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", K(ret), KP(ctx_mgr), K(tx_id)); @@ -52,13 +51,13 @@ int ObCtxTxData::init(ObLSTxCtxMgr *ctx_mgr, int64_t tx_id) } else if (OB_ISNULL(tx_table = ctx_mgr_->get_tx_table())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "tx table is null", KR(ret), K(ctx_mgr_->get_ls_id()), K(*this)); - } else if (OB_FAIL(tx_table->alloc_tx_data(tx_data_))) { + } else if (OB_FAIL(tx_table->alloc_tx_data(tx_data_guard_))) { TRANS_LOG(WARN, "get tx data failed", KR(ret), K(ctx_mgr_->get_ls_id())); - } else if (OB_ISNULL(tx_data_)) { + } else if (OB_ISNULL(tx_data_guard_.tx_data())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "tx data is unexpected null", KR(ret), K(ctx_mgr_->get_ls_id())); } else { - tx_data_->tx_id_ = tx_id; + tx_data_guard_.tx_data()->tx_id_ = tx_id; } } return ret; @@ -67,26 +66,14 @@ int ObCtxTxData::init(ObLSTxCtxMgr *ctx_mgr, int64_t tx_id) void ObCtxTxData::reset() { ctx_mgr_ = nullptr; - tx_data_ = nullptr; + tx_data_guard_.reset(); read_only_ = false; tx_commit_data_.reset(); } void ObCtxTxData::destroy() { - int ret = OB_SUCCESS; - if (tx_data_ != nullptr) { - ObTxTable *tx_table = nullptr; - GET_TX_TABLE_(tx_table); - if (OB_FAIL(ret)) { - - } else if (OB_FAIL(free_tx_data_(tx_table, tx_data_))) { - TRANS_LOG(WARN, "free tx data failed", K(ret), K(*this)); - - } else { - tx_data_ = nullptr; - } - } + reset(); } int ObCtxTxData::insert_into_tx_table() @@ -105,14 +92,12 @@ int ObCtxTxData::insert_into_tx_table() if (OB_FAIL(ret)) { } else { tg.click(); - tx_commit_data_ = *tx_data_; - if (OB_FAIL(insert_tx_data_(tx_table, tx_data_))) { + tx_commit_data_ = *(tx_data_guard_.tx_data()); + if (OB_FAIL(insert_tx_data_(tx_table, tx_data_guard_.tx_data()))) { TRANS_LOG(WARN, "insert tx data failed", K(ret), K(*this)); } else { tg.click(); read_only_ = true; - tx_data_ = NULL; - tx_commit_data_.is_in_tx_data_table_ = true; } } } @@ -123,24 +108,27 @@ int ObCtxTxData::insert_into_tx_table() return ret; } -int ObCtxTxData::recover_tx_data(const ObTxData *tmp_tx_data) +int ObCtxTxData::recover_tx_data(const ObTxData &tmp_tx_data) { int ret = OB_SUCCESS; WLockGuard guard(lock_); + ObTxTable *tx_table = nullptr; + GET_TX_TABLE_(tx_table); - if (OB_FAIL(check_tx_data_writable_())) { + if (OB_FAIL(ret)) { + } else if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret), K(*this)); - } else if (OB_ISNULL(tmp_tx_data)) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid argument", K(ret), KP(tmp_tx_data)); + } else if (OB_FAIL(tx_table->alloc_tx_data(tx_data_guard_))) { + TRANS_LOG(WARN, "alloc tx data failed", KR(ret), K(tmp_tx_data)); } else { - *tx_data_ = *tmp_tx_data; + ObTxData *tx_data = tx_data_guard_.tx_data(); + *tx_data = tmp_tx_data; } return ret; } -int ObCtxTxData::replace_tx_data(ObTxData *&tmp_tx_data) +int ObCtxTxData::replace_tx_data(ObTxData *tmp_tx_data) { int ret = OB_SUCCESS; WLockGuard guard(lock_); @@ -153,17 +141,16 @@ int ObCtxTxData::replace_tx_data(ObTxData *&tmp_tx_data) } else if (OB_ISNULL(tmp_tx_data)) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", K(ret), KP(tmp_tx_data)); - } else if (OB_FAIL(free_tx_data_(tx_table, tx_data_))) { - TRANS_LOG(WARN, "free tx data failed", K(ret), K(*this)); } else { - tx_data_ = tmp_tx_data; - tmp_tx_data = nullptr; + 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)); + } } - return ret; } -int ObCtxTxData::deep_copy_tx_data_out(ObTxData *&tmp_tx_data) +int ObCtxTxData::deep_copy_tx_data_out(ObTxDataGuard &tmp_tx_data_guard) { int ret = OB_SUCCESS; RLockGuard guard(lock_); @@ -174,9 +161,9 @@ int ObCtxTxData::deep_copy_tx_data_out(ObTxData *&tmp_tx_data) ObTxTable *tx_table = nullptr; GET_TX_TABLE_(tx_table) if (OB_FAIL(ret)) { - } else if (OB_FAIL(deep_copy_tx_data_(tx_table, tmp_tx_data))) { - TRANS_LOG(WARN, "deep copy tx data failed", K(ret), KPC(tmp_tx_data), K(*this)); - } else if (OB_ISNULL(tmp_tx_data)) { + } else if (OB_FAIL(deep_copy_tx_data_(tx_table, tmp_tx_data_guard))) { + TRANS_LOG(WARN, "deep copy tx data failed", K(ret), K(tmp_tx_data_guard), K(*this)); + } else if (OB_ISNULL(tmp_tx_data_guard.tx_data())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "copied tmp tx data is null", KR(ret), K(*this)); } @@ -185,9 +172,8 @@ int ObCtxTxData::deep_copy_tx_data_out(ObTxData *&tmp_tx_data) return ret; } -int ObCtxTxData::alloc_tmp_tx_data(storage::ObTxData *&tmp_tx_data) +int ObCtxTxData::alloc_tmp_tx_data(storage::ObTxDataGuard &tmp_tx_data_guard) { - int ret = OB_SUCCESS; RLockGuard guard(lock_); @@ -197,7 +183,7 @@ int ObCtxTxData::alloc_tmp_tx_data(storage::ObTxData *&tmp_tx_data) ObTxTable *tx_table = nullptr; GET_TX_TABLE_(tx_table) if (OB_FAIL(ret)) { - } else if (OB_FAIL(tx_table->alloc_tx_data(tmp_tx_data))) { + } else if (OB_FAIL(tx_table->alloc_tx_data(tmp_tx_data_guard))) { TRANS_LOG(WARN, "alloc tx data failed", K(ret)); } } @@ -212,21 +198,14 @@ int ObCtxTxData::free_tmp_tx_data(ObTxData *&tmp_tx_data) if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret), K(*this)); - } else { - ObTxTable *tx_table = nullptr; - GET_TX_TABLE_(tx_table) - if (OB_FAIL(ret)) { - } else if (OB_FAIL(free_tx_data_(tx_table, tmp_tx_data))) { - TRANS_LOG(WARN, "free tx data failed", K(ret), KPC(tmp_tx_data), K(*this)); - } else { - tmp_tx_data = nullptr; - } + } else if (OB_FAIL(revert_tx_data_(tmp_tx_data))) { + TRANS_LOG(WARN, "free tx data failed", K(ret), KPC(tmp_tx_data), K(*this)); } return ret; } -int ObCtxTxData::insert_tmp_tx_data(ObTxData *&tmp_tx_data) +int ObCtxTxData::insert_tmp_tx_data(ObTxData *tmp_tx_data) { int ret = OB_SUCCESS; RLockGuard guard(lock_); @@ -239,8 +218,6 @@ int ObCtxTxData::insert_tmp_tx_data(ObTxData *&tmp_tx_data) if (OB_FAIL(ret)) { } else if (OB_FAIL(insert_tx_data_(tx_table, tmp_tx_data))) { TRANS_LOG(WARN, "insert tx data failed", K(ret), KPC(tmp_tx_data), K(*this)); - } else { - tmp_tx_data = nullptr; } } @@ -266,7 +243,7 @@ int ObCtxTxData::set_state(int32_t state) if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret), K(*this)); } else { - ATOMIC_STORE(&tx_data_->state_, state); + ATOMIC_STORE(&(tx_data_guard_.tx_data()->state_), state); } return ret; @@ -280,7 +257,7 @@ int ObCtxTxData::set_commit_version(const SCN &commit_version) if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret), K(*this)); } else { - tx_data_->commit_version_.atomic_store(commit_version); + tx_data_guard_.tx_data()->commit_version_.atomic_store(commit_version); } return ret; @@ -295,7 +272,7 @@ int ObCtxTxData::set_start_log_ts(const SCN &start_ts) if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret), K(*this)); } else { - tx_data_->start_scn_.atomic_store(tmp_start_ts); + tx_data_guard_.tx_data()->start_scn_.atomic_store(tmp_start_ts); } return ret; @@ -309,7 +286,7 @@ int ObCtxTxData::set_end_log_ts(const SCN &end_scn) if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret), K(*this)); } else { - tx_data_->end_scn_.atomic_store(end_scn); + tx_data_guard_.tx_data()->end_scn_.atomic_store(end_scn); } return ret; @@ -318,19 +295,22 @@ int ObCtxTxData::set_end_log_ts(const SCN &end_scn) int32_t ObCtxTxData::get_state() const { RLockGuard guard(lock_); - return (NULL != tx_data_ ? ATOMIC_LOAD(&tx_data_->state_): ATOMIC_LOAD(&tx_commit_data_.state_)); + const ObTxData *tx_data = tx_data_guard_.tx_data(); + return (NULL != tx_data ? ATOMIC_LOAD(&tx_data->state_): ATOMIC_LOAD(&tx_commit_data_.state_)); } const SCN ObCtxTxData::get_commit_version() const { RLockGuard guard(lock_); - return (NULL != tx_data_ ? tx_data_->commit_version_.atomic_load() : tx_commit_data_.commit_version_.atomic_load()); + const ObTxData *tx_data = tx_data_guard_.tx_data(); + return (NULL != tx_data ? tx_data->commit_version_.atomic_load() : tx_commit_data_.commit_version_.atomic_load()); } const SCN ObCtxTxData::get_start_log_ts() const { RLockGuard guard(lock_); - SCN ctx_scn = (NULL != tx_data_ ? tx_data_->start_scn_.atomic_load() : tx_commit_data_.start_scn_.atomic_load()); + const ObTxData *tx_data = tx_data_guard_.tx_data(); + SCN ctx_scn = (NULL != tx_data ? tx_data->start_scn_.atomic_load() : tx_commit_data_.start_scn_.atomic_load()); if (ctx_scn.is_max()) { ctx_scn.reset(); } @@ -340,7 +320,8 @@ const SCN ObCtxTxData::get_start_log_ts() const const SCN ObCtxTxData::get_end_log_ts() const { RLockGuard guard(lock_); - SCN ctx_scn = (NULL != tx_data_ ? tx_data_->end_scn_.atomic_load() : tx_commit_data_.end_scn_.atomic_load()); + const ObTxData *tx_data = tx_data_guard_.tx_data(); + SCN ctx_scn = (NULL != tx_data ? tx_data->end_scn_.atomic_load() : tx_commit_data_.end_scn_.atomic_load()); if (ctx_scn.is_max()) { ctx_scn.reset(); } @@ -350,11 +331,12 @@ const SCN ObCtxTxData::get_end_log_ts() const ObTransID ObCtxTxData::get_tx_id() const { RLockGuard guard(lock_); - return (NULL != tx_data_ ? tx_data_->tx_id_ : tx_commit_data_.tx_id_); + const ObTxData *tx_data = tx_data_guard_.tx_data(); + return (NULL != tx_data ? tx_data->tx_id_ : tx_commit_data_.tx_id_); } int ObCtxTxData::prepare_add_undo_action(ObUndoAction &undo_action, - storage::ObTxData *&tmp_tx_data, + storage::ObTxDataGuard &tmp_tx_data_guard, storage::ObUndoStatusNode *&tmp_undo_status) { int ret = OB_SUCCESS; @@ -375,42 +357,36 @@ int ObCtxTxData::prepare_add_undo_action(ObUndoAction &undo_action, } else if (OB_ISNULL(tmp_undo_status)) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "undo status is null", KR(ret), KPC(this)); - } else if (OB_FAIL(tx_table->deep_copy_tx_data(tx_data_, tmp_tx_data))) { + } else if (OB_FAIL(tx_table->deep_copy_tx_data(tx_data_guard_, tmp_tx_data_guard))) { TRANS_LOG(WARN, "copy tx data fail", K(ret), KPC(this)); - } else if (OB_ISNULL(tmp_tx_data)) { + } else if (OB_ISNULL(tmp_tx_data_guard.tx_data())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "copied tx_data is null", KR(ret), KPC(this)); - } else if (OB_FAIL(tmp_tx_data->add_undo_action(tx_table, undo_action))) { + } else if (OB_FAIL(tmp_tx_data_guard.tx_data()->add_undo_action(tx_table, undo_action))) { TRANS_LOG(WARN, "add undo action fail", K(ret), KPC(this)); } - if (OB_FAIL(ret)) { - if (tmp_undo_status) { - tx_table->get_tx_data_table()->free_undo_status_node(tmp_undo_status); - } - if (tmp_tx_data) { - tx_table->free_tx_data(tmp_tx_data); - } + if (OB_FAIL(ret) && OB_NOT_NULL(tmp_undo_status)) { + tx_table->get_tx_data_table()->free_undo_status_node(tmp_undo_status); } } return ret; } -int ObCtxTxData::cancel_add_undo_action(storage::ObTxData *tmp_tx_data, storage::ObUndoStatusNode *tmp_undo_status) +int ObCtxTxData::cancel_add_undo_action(storage::ObUndoStatusNode *tmp_undo_status) { int ret = OB_SUCCESS; ObTxTable *tx_table = nullptr; GET_TX_TABLE_(tx_table); if (OB_SUCC(ret)) { - tx_table->free_tx_data(tmp_tx_data); ret = tx_table->get_tx_data_table()->free_undo_status_node(tmp_undo_status); } return ret; } -int ObCtxTxData::commit_add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatusNode &tmp_undo_status) +int ObCtxTxData::commit_add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatusNode *tmp_undo_status) { - return add_undo_action(undo_action, &tmp_undo_status); + return add_undo_action(undo_action, tmp_undo_status); } int ObCtxTxData::add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatusNode *tmp_undo_status) @@ -425,7 +401,7 @@ int ObCtxTxData::add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatu GET_TX_TABLE_(tx_table); if (OB_FAIL(ret)) { // do nothing - } else if (OB_FAIL(tx_data_->add_undo_action(tx_table, undo_action, tmp_undo_status))) { + } else if (OB_FAIL(tx_data_guard_.tx_data()->add_undo_action(tx_table, undo_action, tmp_undo_status))) { TRANS_LOG(WARN, "add undo action failed", K(ret), K(undo_action), KP(tmp_undo_status), K(*this)); }; } @@ -436,7 +412,7 @@ int ObCtxTxData::add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatu int ObCtxTxData::Guard::get_tx_data(const ObTxData *&tx_data) const { int ret = OB_SUCCESS; - auto tmp_tx_data = host_.tx_data_; + auto tmp_tx_data = host_.tx_data_guard_.tx_data(); if (NULL == tmp_tx_data) { ret = OB_TRANS_CTX_NOT_EXIST; } else { @@ -456,12 +432,9 @@ int ObCtxTxData::check_tx_data_writable_() { int ret = OB_SUCCESS; - if (OB_ISNULL(tx_data_)) { + if (OB_ISNULL(tx_data_guard_.tx_data())) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(WARN, "tx_data_ is not valid", K(this)); - } else if (tx_data_->is_in_tx_data_table_) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(ERROR, "tx_data has inserted into tx table", K(ret), K(this)); } else if (read_only_) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "try to write a read-only tx_data", K(ret), K(this)); @@ -469,7 +442,7 @@ int ObCtxTxData::check_tx_data_writable_() return ret; } -int ObCtxTxData::insert_tx_data_(ObTxTable *tx_table, ObTxData *&tx_data) +int ObCtxTxData::insert_tx_data_(ObTxTable *tx_table, ObTxData *tx_data) { int ret = OB_SUCCESS; @@ -481,43 +454,38 @@ int ObCtxTxData::insert_tx_data_(ObTxTable *tx_table, ObTxData *&tx_data) // no need to insert, do nothing } else if (OB_FAIL(tx_table->insert(tx_data))) { TRANS_LOG(WARN, "insert into tx_table failed", K(ret), KPC(tx_data)); - } else { - tx_data = nullptr; } return ret; } -int ObCtxTxData::free_tx_data_(ObTxTable *tx_table, ObTxData *&tx_data) +int ObCtxTxData::revert_tx_data_(ObTxData *&tx_data) { int ret = OB_SUCCESS; - if (OB_ISNULL(tx_table)) { - ret = OB_INVALID_ARGUMENT; - TRANS_LOG(WARN, "invalid argument", K(ret), KP(tx_table)); - } else if (OB_ISNULL(tx_data)) { + if (OB_ISNULL(tx_data)) { TRANS_LOG(INFO, "tx_data is nullptr, no need to free", KP(tx_data), K(*this)); // no need to free, do nothing } else { - tx_table->free_tx_data(tx_data); tx_data = nullptr; } return ret; } -int ObCtxTxData::deep_copy_tx_data_(ObTxTable *tx_table, storage::ObTxData *&tx_data) +int ObCtxTxData::deep_copy_tx_data_(ObTxTable *tx_table, storage::ObTxDataGuard &out_tx_data_guard) { int ret = OB_SUCCESS; if (OB_ISNULL(tx_table)) { ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", K(ret), KP(tx_table)); - } else if (OB_ISNULL(tx_data_)) { + } else if (OB_ISNULL(tx_data_guard_.tx_data())) { TRANS_LOG(INFO, "tx_data_ is nullptr, no need to deep copy tx data", K(*this)); // no need to free, do nothing - } else if (OB_FAIL(tx_table->deep_copy_tx_data(tx_data_, tx_data))) { - TRANS_LOG(WARN, "deep copy tx data failed", K(ret), KPC(tx_data), K(*this)); + } else if (OB_FAIL(tx_table->deep_copy_tx_data(tx_data_guard_, out_tx_data_guard))) { + TRANS_LOG(WARN, "deep copy tx data failed", K(ret), K(tx_data_guard_), K(out_tx_data_guard), K(*this)); } + return ret; } diff --git a/src/storage/tx/ob_ctx_tx_data.h b/src/storage/tx/ob_ctx_tx_data.h index c29b791d7..8c1ecc85e 100644 --- a/src/storage/tx/ob_ctx_tx_data.h +++ b/src/storage/tx/ob_ctx_tx_data.h @@ -38,12 +38,12 @@ public: bool is_read_only() const { return read_only_; } int insert_into_tx_table(); - int recover_tx_data(const storage::ObTxData *tmp_tx_data); - int replace_tx_data(storage::ObTxData *&tmp_tx_data); - int deep_copy_tx_data_out(storage::ObTxData *&tmp_tx_data); - int alloc_tmp_tx_data(storage::ObTxData *&tmp_tx_data); + int recover_tx_data(const storage::ObTxData &tmp_tx_data); + int replace_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); - int insert_tmp_tx_data(storage::ObTxData *&tmp_tx_data); + int insert_tmp_tx_data(storage::ObTxData *tmp_tx_data); void get_tx_table(storage::ObTxTable *&tx_table); @@ -60,15 +60,15 @@ public: ObTransID get_tx_id() const; int prepare_add_undo_action(ObUndoAction &undo_action, - storage::ObTxData *&tmp_tx_data, + storage::ObTxDataGuard &tmp_tx_data_guard, storage::ObUndoStatusNode *&tmp_undo_status); - int cancel_add_undo_action(storage::ObTxData *tmp_tx_data, storage::ObUndoStatusNode *tmp_undo_status); - int commit_add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatusNode &tmp_undo_status); + int cancel_add_undo_action(storage::ObUndoStatusNode *tmp_undo_status); + int commit_add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatusNode *tmp_undo_status); int add_undo_action(ObUndoAction &undo_action, storage::ObUndoStatusNode *tmp_undo_status = NULL); int get_tx_commit_data(const storage::ObTxCommitData *&tx_commit_data) const; - TO_STRING_KV(KP(ctx_mgr_), KPC(tx_data_), K(tx_commit_data_), K(read_only_)); + TO_STRING_KV(KP(ctx_mgr_), K(tx_data_guard_), K(tx_commit_data_), K(read_only_)); public: class Guard { // TODO(yunxing.cyx): remove it friend class ObCtxTxData; @@ -84,27 +84,27 @@ public: void test_init(storage::ObTxData &tx_data, ObLSTxCtxMgr *ctx_mgr) { ctx_mgr_ = ctx_mgr; - tx_data_ = &tx_data; + tx_data_guard_.init(&tx_data); read_only_ = false; } void test_tx_data_reset() { - if (OB_NOT_NULL(tx_data_)) { - tx_data_->reset(); + if (OB_NOT_NULL(tx_data_guard_.tx_data())) { + tx_data_guard_.reset(); } } void test_set_tx_id(int64_t tx_id) { - if (OB_NOT_NULL(tx_data_)) { - tx_data_->tx_id_ = tx_id; + if (OB_NOT_NULL(tx_data_guard_.tx_data())) { + tx_data_guard_.tx_data()->tx_id_ = tx_id; } } private: int check_tx_data_writable_(); - int insert_tx_data_(storage::ObTxTable *tx_table, storage::ObTxData *&tx_data); - int free_tx_data_(storage::ObTxTable *tx_table, storage::ObTxData *&tx_data); - int deep_copy_tx_data_(storage::ObTxTable *tx_table, storage::ObTxData *&tx_data); + int insert_tx_data_(storage::ObTxTable *tx_table, storage::ObTxData *tx_data); + int deep_copy_tx_data_(storage::ObTxTable *tx_table, storage::ObTxDataGuard &tx_data); + int revert_tx_data_(storage::ObTxData *&tx_data); private: typedef common::SpinRWLock RWLock; @@ -112,7 +112,7 @@ private: typedef common::SpinWLockGuard WLockGuard; private: ObLSTxCtxMgr *ctx_mgr_; - storage::ObTxData *tx_data_; + storage::ObTxDataGuard tx_data_guard_; storage::ObTxCommitData tx_commit_data_; bool read_only_; // lock for tx_data_ pointer diff --git a/src/storage/tx/ob_trans_define_v4.h b/src/storage/tx/ob_trans_define_v4.h index 4554c3689..d48a8582a 100644 --- a/src/storage/tx/ob_trans_define_v4.h +++ b/src/storage/tx/ob_trans_define_v4.h @@ -560,6 +560,7 @@ public: void set_can_elr(const bool can_elr) { can_elr_ = can_elr; } bool is_can_elr() const { return can_elr_; } bool need_rollback() { return state_ == State::ABORTED; } + share::SCN get_snapshot_version() { return snapshot_version_; } ObITxCallback *cancel_commit_cb(); }; diff --git a/src/storage/tx/ob_trans_part_ctx.cpp b/src/storage/tx/ob_trans_part_ctx.cpp index ded5e6a9a..874f6040f 100644 --- a/src/storage/tx/ob_trans_part_ctx.cpp +++ b/src/storage/tx/ob_trans_part_ctx.cpp @@ -124,6 +124,7 @@ int ObPartTransCtx::init(const uint64_t tenant_id, epoch_ = epoch; pending_write_ = 0; set_role_state(for_replay); + block_frozen_memtable_ = nullptr; if (is_follower_()) { mt_ctx_.trans_replay_begin(); @@ -232,6 +233,7 @@ void ObPartTransCtx::destroy() timeout_task_.destroy(); clog_encrypt_info_.destroy(); trace_info_.reset(); + block_frozen_memtable_ = nullptr; is_inited_ = false; } } @@ -1285,7 +1287,7 @@ int ObPartTransCtx::check_scheduler_status() return OB_SUCCESS; } -int ObPartTransCtx::recover_tx_ctx_table_info(const ObTxCtxTableInfo &ctx_info) +int ObPartTransCtx::recover_tx_ctx_table_info(ObTxCtxTableInfo &ctx_info) { int ret = OB_SUCCESS; CtxLockGuard guard(lock_); @@ -1297,7 +1299,7 @@ int ObPartTransCtx::recover_tx_ctx_table_info(const 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.state_info_))) { + } else if (OB_FAIL(ctx_tx_data_.recover_tx_data(ctx_info.state_info_))) { TRANS_LOG(WARN, "recover tx data failed", K(ret), K(ctx_tx_data_)); } else { trans_id_ = ctx_info.tx_id_; @@ -1478,28 +1480,31 @@ int ObPartTransCtx::submit_redo_log(const bool is_freeze) bool final_log_submitting = false; if (is_freeze) { - // spin lock - CtxLockGuard guard(lock_); - tg.click(); - ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, true); + bool need_submit = !is_logging_blocked(); + if (need_submit) { + // spin lock + CtxLockGuard guard(lock_); + tg.click(); + ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, true); - is_tx_committing = ObTxState::INIT != get_downstream_state(); - final_log_submitting = final_log_cb_.is_valid(); - if (!is_tx_committing && !final_log_submitting) { - (void)mt_ctx_.merge_multi_callback_lists_for_immediate_logging(); - ret = submit_log_impl_(ObTxLogType::TX_REDO_LOG); - if (OB_SUCC(ret) || OB_BLOCK_FROZEN == ret) { - ret = submit_multi_data_source_(); + is_tx_committing = ObTxState::INIT != get_downstream_state(); + final_log_submitting = final_log_cb_.is_valid(); + if (!is_tx_committing && !final_log_submitting) { + (void)mt_ctx_.merge_multi_callback_lists_for_immediate_logging(); + ret = submit_log_impl_(ObTxLogType::TX_REDO_LOG); + if (OB_SUCC(ret) || OB_BLOCK_FROZEN == ret) { + ret = submit_multi_data_source_(); + } + try_submit = true; + } + tg.click(); + ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, false); + 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(ctx_ref), get_ref()); } - try_submit = true; - } - tg.click(); - ATOMIC_STORE(&is_submitting_redo_log_for_freeze_, false); - 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(ctx_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) { @@ -1508,19 +1513,16 @@ int ObPartTransCtx::submit_redo_log(const bool is_freeze) TRANS_LOG(ERROR, "try lock error, unexpected error", K(ret), K(*this)); } } else { - CtxLockGuard guard(lock_, false); tg.click(); - if (mt_ctx_.pending_log_size_too_large()) { - is_tx_committing = ObTxState::INIT != get_downstream_state(); - final_log_submitting = final_log_cb_.is_valid(); + is_tx_committing = ObTxState::INIT != get_downstream_state(); + final_log_submitting = final_log_cb_.is_valid(); - if (!is_tx_committing && !final_log_submitting) { - (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) { + (void)mt_ctx_.merge_multi_callback_lists_for_immediate_logging(); + ret = submit_log_impl_(ObTxLogType::TX_REDO_LOG); + try_submit = true; } tg.click(); if (try_submit) { @@ -1531,11 +1533,36 @@ int ObPartTransCtx::submit_redo_log(const bool is_freeze) } if (OB_BLOCK_FROZEN == ret) { ret = OB_SUCCESS; + } else { + clear_block_frozen_memtable(); } return ret; } +int ObPartTransCtx::set_block_frozen_memtable(memtable::ObMemtable *memtable) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(memtable)) { + ret = OB_ERR_UNEXPECTED; + TRANS_LOG(WARN, "memtable cannot be null", K(ret), KPC(this)); + } else { + ATOMIC_STORE(&block_frozen_memtable_, memtable); + } + return ret; +} + +void ObPartTransCtx::clear_block_frozen_memtable() +{ + ATOMIC_STORE(&block_frozen_memtable_, nullptr); +} + +bool ObPartTransCtx::is_logging_blocked() +{ + memtable::ObMemtable *memtable = ATOMIC_LOAD(&block_frozen_memtable_); + return OB_NOT_NULL(memtable) && memtable->get_logging_blocked(); +} + void ObPartTransCtx::get_audit_info(int64_t &lock_for_read_elapse) const { lock_for_read_elapse = mt_ctx_.get_lock_for_read_elapse(); @@ -5597,34 +5624,21 @@ int ObPartTransCtx::supplement_undo_actions_if_exist_() int ret = OB_SUCCESS; ObTxTable *tx_table = nullptr; - ObTxData *tmp_tx_data = nullptr; + ObTxDataGuard tmp_tx_data_guard; + tmp_tx_data_guard.reset(); ctx_tx_data_.get_tx_table(tx_table); - if (OB_FAIL(ctx_tx_data_.deep_copy_tx_data_out(tmp_tx_data))) { + 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)); - } else if (OB_FAIL(tx_table->supplement_undo_actions_if_exist(tmp_tx_data))) { + } else if (OB_FAIL(tx_table->supplement_undo_actions_if_exist(tmp_tx_data_guard.tx_data()))) { TRANS_LOG( 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))) { + } else if (OB_FAIL(ctx_tx_data_.replace_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)); } - // tmp_tx_data should be null because replace_tx_data should set it as nullptr if succeed. - // we free it if not null to avoid memory leak - if (OB_NOT_NULL(tmp_tx_data)) { - if (OB_SUCC(ret)) { - ret = OB_ERR_UNEXPECTED; - TRANS_LOG(WARN, "unexpected error happend when supplement undo actions", KR(ret), - K(ctx_tx_data_), KPC(this)); - } - int tmp_ret = ctx_tx_data_.free_tmp_tx_data(tmp_tx_data); - TRANS_LOG(WARN, "error happend when supplement undo actions", KR(ret), KR(tmp_ret), - K(ctx_tx_data_), KPC(this)); - - } - return ret; } @@ -5825,34 +5839,30 @@ int ObPartTransCtx::rollback_to_savepoint_(const int64_t from_scn, * 2. add UndoAction into tx_ctx's tx_data * 3. insert UndoAction into tx_data_table after log sync success */ + ObTxDataGuard tmp_tx_data_guard; + tmp_tx_data_guard.reset(); if (is_follower_()) { /* Follower */ ObUndoAction undo_action(from_scn, to_scn); - ObTxData *tmp_tx_data = nullptr; 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))) { + } 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->end_scn_ = exec_info_.max_applying_log_ts_)) { - } else if (OB_FAIL(ctx_tx_data_.insert_tmp_tx_data(tmp_tx_data))) { + } else if (FALSE_IT(tmp_tx_data_guard.tx_data()->end_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)); - int tmp_ret = OB_SUCCESS; - if (OB_TMP_FAIL(ctx_tx_data_.free_tmp_tx_data(tmp_tx_data))) { - TRANS_LOG(WARN, "free tmp tx data failed", KR(tmp_ret), KPC(this)); - } } } else if (OB_UNLIKELY(exec_info_.max_submitted_seq_no_ > to_scn)) { /* Leader */ ObUndoAction undo_action(from_scn, to_scn); ObUndoStatusNode *undo_status = NULL; - ObTxData *tmp_tx_data = NULL; - if (OB_FAIL(ctx_tx_data_.prepare_add_undo_action(undo_action, tmp_tx_data, undo_status))) { + if (OB_FAIL(ctx_tx_data_.prepare_add_undo_action(undo_action, tmp_tx_data_guard, undo_status))) { TRANS_LOG(WARN, "prepare add undo action fail", K(ret), KPC(this)); - } else if (OB_FAIL(submit_rollback_to_log_(from_scn, to_scn, tmp_tx_data))) { + } else if (OB_FAIL(submit_rollback_to_log_(from_scn, to_scn, tmp_tx_data_guard.tx_data()))) { TRANS_LOG(WARN, "submit undo redolog fail", K(ret), K(from_scn), K(to_scn), KPC(this)); int tmp_ret = OB_SUCCESS; - if (OB_TMP_FAIL(ctx_tx_data_.cancel_add_undo_action(tmp_tx_data, undo_status))) { + if (OB_TMP_FAIL(ctx_tx_data_.cancel_add_undo_action(undo_status))) { TRANS_LOG(ERROR, "cancel add undo action failed", KR(tmp_ret), KPC(this)); } - } else if (OB_FAIL(ctx_tx_data_.commit_add_undo_action(undo_action, *undo_status))) { + } else if (OB_FAIL(ctx_tx_data_.commit_add_undo_action(undo_action, undo_status))) { TRANS_LOG(ERROR, "oops, commit add undo action fail", K(ret), KPC(this)); ob_abort(); } diff --git a/src/storage/tx/ob_trans_part_ctx.h b/src/storage/tx/ob_trans_part_ctx.h index 5130bec25..e19b2d794 100644 --- a/src/storage/tx/ob_trans_part_ctx.h +++ b/src/storage/tx/ob_trans_part_ctx.h @@ -344,7 +344,7 @@ public: // get_tx_ctx_table_info returns OB_TRANS_CTX_NOT_EXIST if the tx ctx table need not to be // dumped. int get_tx_ctx_table_info(ObTxCtxTableInfo &info); - int recover_tx_ctx_table_info(const ObTxCtxTableInfo &ctx_info); + int recover_tx_ctx_table_info(ObTxCtxTableInfo &ctx_info); // leader switch related bool need_callback_scheduler_(); @@ -503,7 +503,6 @@ private: int get_prev_log_lsn_(const ObTxLogBlock &log_block, ObTxLogType prev_log_type, palf::LSN &lsn); // int init_tx_data_(const share::ObLSID&ls_id, const ObTransID &tx_id); - // int free_tx_data_(ObTxData *&tx_data); bool is_local_tx_() const { return TransType::SP_TRANS == exec_info_.trans_type_; } void set_trans_type_(int64_t trans_type) { exec_info_.trans_type_ = trans_type; } @@ -657,6 +656,9 @@ public: */ int end_access(); int rollback_to_savepoint(const int64_t op_sn, const int64_t from_scn, const int64_t to_scn); + int set_block_frozen_memtable(memtable::ObMemtable *memtable); + void clear_block_frozen_memtable(); + bool is_logging_blocked(); private: int check_status_(); int tx_keepalive_response_(const int64_t status); @@ -721,6 +723,9 @@ private: common::ObDList free_cbs_; common::ObDList busy_cbs_; ObTxLogCb final_log_cb_; + // flag if the first callback is linked to a logging_block memtable + // to prevent unnecessary submit_log actions for freeze + memtable::ObMemtable *block_frozen_memtable_; // The semantic of the rec_log_ts means the log ts of the first state change // after the previous checkpoint. So we use the current strategy to maintain // the rec_log_ts: diff --git a/src/storage/tx/ob_trans_service.cpp b/src/storage/tx/ob_trans_service.cpp index 2bf76f8e9..efb8407d0 100644 --- a/src/storage/tx/ob_trans_service.cpp +++ b/src/storage/tx/ob_trans_service.cpp @@ -814,6 +814,8 @@ int ObTransService::register_mds_into_ctx(ObTxDesc &tx_desc, ObStoreCtx store_ctx; ObTxReadSnapshot snapshot; snapshot.init_none_read(); + concurrent_control::ObWriteFlag write_flag; + write_flag.set_is_mds(); if (OB_UNLIKELY(!tx_desc.is_valid() || !ls_id.is_valid() || OB_ISNULL(buf) || @@ -821,7 +823,7 @@ int ObTransService::register_mds_into_ctx(ObTxDesc &tx_desc, ret = OB_INVALID_ARGUMENT; TRANS_LOG(WARN, "invalid argument", KR(ret), K(tx_desc), K(ls_id), KP(buf), K(buf_len)); } else if (FALSE_IT(store_ctx.ls_id_ = ls_id)) { - } else if (OB_FAIL(get_write_store_ctx(tx_desc, snapshot, store_ctx))) { + } else if (OB_FAIL(get_write_store_ctx(tx_desc, snapshot, write_flag, store_ctx))) { TRANS_LOG(WARN, "get store ctx failed", KR(ret), K(tx_desc), K(ls_id)); } else { do { diff --git a/src/storage/tx/ob_trans_service_v4.cpp b/src/storage/tx/ob_trans_service_v4.cpp index 70ba5e9ee..6359268a9 100644 --- a/src/storage/tx/ob_trans_service_v4.cpp +++ b/src/storage/tx/ob_trans_service_v4.cpp @@ -998,6 +998,7 @@ int ObTransService::get_read_store_ctx(const SCN snapshot_version, int ObTransService::get_write_store_ctx(ObTxDesc &tx, const ObTxReadSnapshot &snapshot, + const concurrent_control::ObWriteFlag write_flag, storage::ObStoreCtx &store_ctx) { int ret = OB_SUCCESS; @@ -1045,7 +1046,8 @@ int ObTransService::get_write_store_ctx(ObTxDesc &tx, tx_table_guard, snap, store_ctx.timeout_, - tx.lock_timeout_us_ + tx.lock_timeout_us_, + write_flag ); if (tx.get_active_ts() <= 0) { tx.active_ts_ = ObClockGenerator::getClock(); diff --git a/src/storage/tx/ob_trans_service_v4.h b/src/storage/tx/ob_trans_service_v4.h index 8219e669c..4f8393a4a 100644 --- a/src/storage/tx/ob_trans_service_v4.h +++ b/src/storage/tx/ob_trans_service_v4.h @@ -78,6 +78,7 @@ int get_read_store_ctx(const share::SCN snapshot_version, ObStoreCtx &store_ctx); int get_write_store_ctx(ObTxDesc &tx, const ObTxReadSnapshot &snapshot, + const concurrent_control::ObWriteFlag write_flag, storage::ObStoreCtx &store_ctx); int revert_store_ctx(storage::ObStoreCtx &store_ctx); diff --git a/src/storage/tx/ob_trans_submit_log_cb.cpp b/src/storage/tx/ob_trans_submit_log_cb.cpp index 6c6db1d01..221cb6f44 100644 --- a/src/storage/tx/ob_trans_submit_log_cb.cpp +++ b/src/storage/tx/ob_trans_submit_log_cb.cpp @@ -83,7 +83,7 @@ int ObTxLogCb::init(const ObLSID &key, ls_id_ = key; trans_id_ = trans_id; ctx_ = ctx; - tx_data_ = nullptr; + tx_data_guard_.reset(); is_callbacked_ = false; } @@ -98,7 +98,7 @@ void ObTxLogCb::reset() ls_id_.reset(); trans_id_.reset(); ctx_ = NULL; - tx_data_ = nullptr; + tx_data_guard_.reset(); callbacks_.reset(); is_callbacked_ = false; cb_arg_array_.reset(); @@ -109,7 +109,7 @@ void ObTxLogCb::reset() void ObTxLogCb::reuse() { ObTxBaseLogCb::reuse(); - tx_data_ = nullptr; + tx_data_guard_.reset(); callbacks_.reset(); is_callbacked_ = false; cb_arg_array_.reset(); diff --git a/src/storage/tx/ob_trans_submit_log_cb.h b/src/storage/tx/ob_trans_submit_log_cb.h index 9f4f49307..62dbd8508 100644 --- a/src/storage/tx/ob_trans_submit_log_cb.h +++ b/src/storage/tx/ob_trans_submit_log_cb.h @@ -80,8 +80,15 @@ public: void destroy() { reset(); } ObTxLogType get_last_log_type() const; ObTransCtx *get_ctx() { return ctx_; } - void set_tx_data(ObTxData *tx_data) { tx_data_ = tx_data; } - ObTxData* get_tx_data() { return tx_data_; } + void set_tx_data(ObTxData *tx_data) + { + if (OB_ISNULL(tx_data)) { + tx_data_guard_.reset(); + } else { + tx_data_guard_.init(tx_data); + } + } + 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; } @@ -103,7 +110,7 @@ public: K_(trans_id), K_(ls_id), KP_(ctx), - KP_(tx_data), + K_(tx_data_guard), K(is_callbacked_), K(mds_range_), K(cb_arg_array_)); @@ -116,7 +123,7 @@ private: share::ObLSID ls_id_; ObTransID trans_id_; ObTransCtx *ctx_; - ObTxData *tx_data_; + ObTxDataGuard tx_data_guard_; memtable::ObCallbackScope callbacks_; bool is_callbacked_; ObTxMDSRange mds_range_; diff --git a/src/storage/tx/ob_tx_data_define.cpp b/src/storage/tx/ob_tx_data_define.cpp index da1e5368f..7257a7bda 100644 --- a/src/storage/tx/ob_tx_data_define.cpp +++ b/src/storage/tx/ob_tx_data_define.cpp @@ -221,7 +221,6 @@ void ObTxCommitData::reset() commit_version_.reset(); start_scn_.reset(); end_scn_.reset(); - is_in_tx_data_table_ = false; } const char* ObTxCommitData::get_state_string(int32_t state) @@ -355,7 +354,15 @@ int ObTxData::deserialize_(const char *buf, void ObTxData::reset() { + if (OB_NOT_NULL(slice_allocator_) || ref_cnt_ != 0) { + int ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "this tx data should not be reset", KR(ret), KP(this), KP(slice_allocator_), K(ref_cnt_)); + // TODO : @gengli remove ob_abort + ob_abort(); + } ObTxCommitData::reset(); + slice_allocator_ = nullptr; + ref_cnt_ = 0; undo_status_list_.reset(); } @@ -372,7 +379,6 @@ ObTxData &ObTxData::operator=(const ObTxData &rhs) start_scn_ = rhs.start_scn_; end_scn_ = rhs.end_scn_; undo_status_list_ = rhs.undo_status_list_; - is_in_tx_data_table_ = rhs.is_in_tx_data_table_; return *this; } @@ -383,7 +389,6 @@ ObTxData &ObTxData::operator=(const ObTxCommitData &rhs) commit_version_ = rhs.commit_version_; start_scn_ = rhs.start_scn_; end_scn_ = rhs.end_scn_; - is_in_tx_data_table_ = rhs.is_in_tx_data_table_; undo_status_list_.reset(); return *this; } @@ -440,7 +445,6 @@ int ObTxData::add_undo_action(ObTxTable *tx_table, transaction::ObUndoAction &ne 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) { - // STORAGE_LOG(DEBUG, "generate new undo status node"); ObUndoStatusNode *new_node = nullptr; if (OB_NOT_NULL(undo_node)) { new_node = undo_node; @@ -448,6 +452,7 @@ int ObTxData::add_undo_action(ObTxTable *tx_table, transaction::ObUndoAction &ne } else if (OB_FAIL(tx_data_table->alloc_undo_status_node(new_node))) { STORAGE_LOG(WARN, "alloc_undo_status_node() fail", KR(ret)); } + if (OB_SUCC(ret)) { new_node->next_ = node; undo_status_list_.head_ = new_node; @@ -455,13 +460,7 @@ int ObTxData::add_undo_action(ObTxTable *tx_table, transaction::ObUndoAction &ne undo_status_list_.undo_node_cnt_++; } } - node->undo_actions_[node->size_++] = new_undo_action; - if (undo_status_list_.undo_node_cnt_ * TX_DATA_SLICE_SIZE > OB_MAX_TX_SERIALIZE_SIZE) { - ret = OB_SIZE_OVERFLOW; - STORAGE_LOG(WARN, "Too many undo actions. The size of tx data is overflow.", KR(ret), - K(undo_status_list_.undo_node_cnt_), KPC(this)); - } } if (OB_NOT_NULL(undo_node)) { @@ -574,10 +573,9 @@ bool ObTxData::equals_(ObTxData &rhs) void ObTxData::print_to_stderr(const ObTxData &tx_data) { fprintf(stderr, - "TX_DATA:{tx_id=%-20ld in_tx_data_table=%-6s start_log_scn=%-20s end_log_scn=%-20s commit_version=%-20s " + "TX_DATA:{tx_id=%-20ld start_log_scn=%-20s end_log_scn=%-20s commit_version=%-20s " "state=%s", tx_data.tx_id_.get_id(), - tx_data.is_in_tx_data_table_ ? "True" : "False", to_cstring(tx_data.start_scn_), to_cstring(tx_data.end_scn_), to_cstring(tx_data.commit_version_), @@ -593,10 +591,9 @@ void ObTxData::dump_2_text(FILE *fd) const } fprintf(fd, - "TX_DATA:\n{\n tx_id=%-20ld\n in_tx_data_table=%-6s\n start_log_scn=%-20s\n end_log_scn=%-20s\n " + "TX_DATA:\n{\n tx_id=%-20ld\n start_log_scn=%-20s\n end_log_scn=%-20s\n " " commit_version=%-20s\n state=%s\n", tx_id_.get_id(), - is_in_tx_data_table_ ? "True" : "False", to_cstring(start_scn_), to_cstring(end_scn_), to_cstring(commit_version_), @@ -612,8 +609,8 @@ DEF_TO_STRING(ObTxData) int64_t pos = 0; J_OBJ_START(); J_KV(K_(tx_id), + K_(ref_cnt), "state", get_state_string(state_), - "in_tx_data_table", is_in_tx_data_table_ ? "True" : "False", K_(commit_version), K_(start_scn), K_(end_scn), @@ -622,35 +619,6 @@ DEF_TO_STRING(ObTxData) return pos; } -TxDataHashNode *TxDataHashMapAllocHandle::alloc_node(ObTxData *tx_data) -{ - void *hash_node_ptr = reinterpret_cast(ObTxData::get_hash_node_by_tx_data(tx_data)); - return new (hash_node_ptr) TxDataHashNode(); -} -void TxDataHashMapAllocHandle::free_node(TxDataHashNode *node) -{ - if (nullptr != node) { - // free undo status node first - ObTxData *tx_data = ObTxData::get_tx_data_by_hash_node(node); - free_undo_list_(tx_data->undo_status_list_.head_); - tx_data->undo_status_list_.head_ = nullptr; - - // free slice memory - void *slice_ptr = reinterpret_cast(node); - slice_allocator_->free(slice_ptr); - } -} - -void TxDataHashMapAllocHandle::free_undo_list_(ObUndoStatusNode *node_ptr) -{ - ObUndoStatusNode *node_to_free = nullptr; - while (nullptr != node_ptr) { - node_to_free = node_ptr; - node_ptr = node_ptr->next_; - slice_allocator_->free(reinterpret_cast(node_to_free)); - } -} - DEF_TO_STRING(ObUndoStatusNode) { int64_t pos = 0; diff --git a/src/storage/tx/ob_tx_data_define.h b/src/storage/tx/ob_tx_data_define.h index f862c9d79..0b40008bf 100644 --- a/src/storage/tx/ob_tx_data_define.h +++ b/src/storage/tx/ob_tx_data_define.h @@ -15,11 +15,12 @@ #include "lib/allocator/ob_slice_alloc.h" #include "share/scn.h" +#include "lib/objectpool/ob_server_object_pool.h" #include "storage/tx/ob_committer_define.h" #include "storage/tx/ob_trans_define.h" +#include "storage/tx_table/ob_tx_data_hash_map.h" #include "storage/ob_i_table.h" - namespace oceanbase { @@ -31,7 +32,6 @@ class ObTxTable; class ObTxDataTable; class ObTxDataMemtable; class ObTxDataMemtableMgr; -class TxDataHashMapAllocHandle; // The memory structures associated with tx data are shown below. They are designed for several // reasons: @@ -48,19 +48,18 @@ class TxDataHashMapAllocHandle; // // // A Piece of Memory Slice -// ------------------------------> +-------------------------+ +----------------+ -// | | | | -// TX_DATA_HASH_NODE_SIZE | TxDataHashNode | | | -// | | | | -// ------------------------------> +-------------------------+ +----------------+ -// | | | | -// | | | | -// TX_DATA_SIZE | ObTxData | | | -// | | | | -// | | | | -// ------------------------------> +-------------------------+ +----->+----------------| +// ------------------------------> +-------------------------+ +----->+----------------+ // | | | | | -// TX_DATA_SORT_LIST_NODE_SIZE | TxDataSortListNode | | | | +// | | | | | +// | | | | | +// | | | | | +// | | | | | +// TX_DATA_SLICE_SIZE | ObTxData | | | | +// | | | | | +// | | | | | +// | | | +----------------| +// | | | | | +// | TxDataSortListNode | | | | // | (*next) |------+ | | // ------------------------------> +-------------------------+ +----------------+ // @@ -92,41 +91,13 @@ class TxDataHashMapAllocHandle; // +-------------------------+ +----------------+ // // -// The third kind of slice is almost identical to the first. It used in tx data sstable cache. The -// lastest_used_time_stamp is used to clean the tx data sstable cache periodically. -// -// A Piece of Memory Slice -// ------------------------------> +-------------------------+ -// | | -// TX_DATA_HASH_NODE_SIZE | TxDataHashNode | -// | | -// ------------------------------> +-------------------------+ -// | | -// | | -// TX_DATA_SIZE | ObTxData | -// | | -// | | -// ------------------------------> +-------------------------+ -// | | -// TX_DATA_SORT_LIST_NODE_SIZE | latest_used_time_stamp | -// | | -// ------------------------------> +-------------------------+ -// -static const int TX_DATA_HASH_NODE_SIZE = 56; -static const int TX_DATA_SIZE = 72; -static const int TX_DATA_SORT_LIST_NODE_SIZE = 8; -static const int TX_DATA_SLICE_SIZE = 136; -static const int TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE = (TX_DATA_SLICE_SIZE / 16) - 1; -static const int TX_DATA_OFFSET_BETWEEN_DATA_AND_SORT_NODE - = TX_DATA_SLICE_SIZE - TX_DATA_HASH_NODE_SIZE - TX_DATA_SORT_LIST_NODE_SIZE; -// Reserve 5KB to store the fields in tx data except undo_status -static const int OB_MAX_TX_SERIALIZE_SIZE = OB_MAX_USER_ROW_LENGTH - 5120; +static const int TX_DATA_SLICE_SIZE = 128; +static const int UNDO_ACTION_SZIE = 16; +static const int TX_DATA_UNDO_ACT_MAX_NUM_PER_NODE = (TX_DATA_SLICE_SIZE / UNDO_ACTION_SZIE) - 1; static const int MAX_TX_DATA_MEMTABLE_CNT = 2; -using TxDataHashNode = common::LinkHashNode; -using TxDataHashValue = common::LinkHashValue; -using TxDataMap = common::ObLinkHashMap; +using TxDataMap = ObTxDataHashMap; // DONT : Modify this definition struct ObUndoStatusNode @@ -138,11 +109,11 @@ struct ObUndoStatusNode ObUndoStatusNode() : size_(0), next_(nullptr) {} }; -struct ObTxDataSortListNode +struct ObTxDataLinkNode { - struct ObTxDataSortListNode* next_; + ObTxData* next_; - ObTxDataSortListNode() : next_(nullptr) {} + ObTxDataLinkNode() : next_(nullptr) {} void reset() { next_ = nullptr; } TO_STRING_KV(KP_(next)); @@ -212,7 +183,6 @@ public: void reset(); TO_STRING_KV(K_(tx_id), K_(state), - K_(is_in_tx_data_table), K_(commit_version), K_(start_scn), K_(end_scn)); @@ -231,24 +201,64 @@ public: public: transaction::ObTransID tx_id_; int32_t state_; - bool is_in_tx_data_table_; share::SCN commit_version_; share::SCN start_scn_; share::SCN end_scn_; }; -class ObTxData : public ObTxCommitData, public TxDataHashValue + +class ObTxDataLink +{ +public: + ObTxDataLink() : sort_list_node_(), hash_node_() {} + // used for mini merge + ObTxDataLinkNode sort_list_node_; + // used for hash conflict + ObTxDataLinkNode hash_node_; +}; + +// DONT : Modify this definition +class ObTxData : public ObTxCommitData, public ObTxDataLink { - friend TxDataHashMapAllocHandle; private: const static int64_t UNIS_VERSION = 1; public: - ObTxData() { reset(); } + ObTxData() : ObTxCommitData(), ObTxDataLink(), slice_allocator_(nullptr), ref_cnt_(0), undo_status_list_(), flag_(0) {} ObTxData(const ObTxData &rhs); ObTxData &operator=(const ObTxData &rhs); ObTxData &operator=(const ObTxCommitData &rhs); ~ObTxData() {} void reset(); + OB_INLINE bool contain(const transaction::ObTransID &tx_id) { return tx_id_ == tx_id; } + + int64_t inc_ref() + { + int64_t ref_cnt = ATOMIC_AAF(&ref_cnt_, 1); + return ref_cnt; + } + + void dec_ref() + { +#ifdef UNITTEST + return; +#endif + if (nullptr == slice_allocator_) { + STORAGE_LOG(ERROR, "invalid slice allocator", KPC(this)); + ob_abort(); + } else if (0 == ATOMIC_SAF(&ref_cnt_, 1)) { + if (OB_UNLIKELY(nullptr != undo_status_list_.head_)) { + ObUndoStatusNode *node_ptr = undo_status_list_.head_; + ObUndoStatusNode *node_to_free = nullptr; + while (nullptr != node_ptr) { + node_to_free = node_ptr; + node_ptr = node_ptr->next_; + slice_allocator_->free(node_to_free); + } + } + slice_allocator_->free(this); + } + } + /** * @brief Add a undo action with dynamically memory allocation. * See more details in alloc_undo_status_node() function of class ObTxDataTable @@ -257,7 +267,9 @@ public: * @param[in & out] undo_action, the undo action which is waiting to be added. If this undo action contains exsiting undo actions, the existing undo actions will be deleted and this undo action will be modified to contain all the deleted undo actions. * @param[in] undo_node, the undo status node can be used to extend undo status list if required, otherwise it will be released */ - int add_undo_action(ObTxTable *tx_table, transaction::ObUndoAction &undo_action, ObUndoStatusNode *undo_node = nullptr); + OB_NOINLINE int add_undo_action(ObTxTable *tx_table, + transaction::ObUndoAction &undo_action, + ObUndoStatusNode *undo_node = nullptr); /** * @brief Check if this tx data is valid */ @@ -277,148 +289,72 @@ private: int64_t get_serialize_size_() const; bool equals_(ObTxData &rhs); int merge_undo_actions_(ObTxDataTable *tx_data_table, - ObUndoStatusNode *&node, - transaction::ObUndoAction &undo_action); + ObUndoStatusNode *&node, + transaction::ObUndoAction &undo_action); public: - /** - * @brief The latest used time stamp use the same memeory as the sort list node. This function - * only used when reading tx data in sstable cache because we need update the time stamp to decide - * which tx data should be deleted. - * - * @param tx_data the pointer of tx data - * @return int64_t* the pointer of latest used time stamp - */ - OB_INLINE static int64_t *get_latest_used_ts_by_tx_data(ObTxData *tx_data) { - if (nullptr == tx_data) { - return nullptr; - } - char *sort_list_node_char_ptr - = reinterpret_cast(tx_data) + TX_DATA_OFFSET_BETWEEN_DATA_AND_SORT_NODE; - int64_t *latest_use_ts - = reinterpret_cast(sort_list_node_char_ptr); - return latest_use_ts; - } - OB_INLINE static ObTxDataSortListNode *get_sort_list_node_by_tx_data(ObTxData *tx_data) - { - if (nullptr == tx_data) { - return nullptr; - } - char *sort_list_node_char_ptr - = reinterpret_cast(tx_data) + TX_DATA_OFFSET_BETWEEN_DATA_AND_SORT_NODE; - ObTxDataSortListNode *sort_list_node - = reinterpret_cast(sort_list_node_char_ptr); - return sort_list_node; - } - - OB_INLINE static TxDataHashNode *get_hash_node_by_tx_data(ObTxData *tx_data) - { - if (nullptr == tx_data) { - return nullptr; - } - char *hash_node_char_ptr = reinterpret_cast(tx_data) - TX_DATA_HASH_NODE_SIZE; - TxDataHashNode *hash_node = reinterpret_cast(hash_node_char_ptr); - return hash_node; - } - - OB_INLINE static ObTxData *get_tx_data_by_hash_node(TxDataHashNode *hash_node) - { - if (nullptr == hash_node) { - return nullptr; - } - char *tx_data_char_ptr = reinterpret_cast(hash_node) + TX_DATA_HASH_NODE_SIZE; - ObTxData *tx_data = reinterpret_cast(tx_data_char_ptr); - return tx_data; - } - - OB_INLINE static ObTxData *get_tx_data_by_sort_list_node(ObTxDataSortListNode *sort_list_node) + OB_INLINE static ObTxData *get_tx_data_by_sort_list_node(ObTxDataLinkNode *sort_list_node) { if (nullptr == sort_list_node) { return nullptr; } - char *tx_data_char_ptr - = reinterpret_cast(sort_list_node) - TX_DATA_OFFSET_BETWEEN_DATA_AND_SORT_NODE; - ObTxData *tx_data = reinterpret_cast(tx_data_char_ptr); + ObTxData *tx_data = static_cast(reinterpret_cast(sort_list_node)); return tx_data; } public: + ObSliceAlloc *slice_allocator_; + int64_t ref_cnt_; ObUndoStatusList undo_status_list_; -}; - - -class TxDataHashMapAllocHandle -{ - using SliceAllocator = ObSliceAlloc; - -public: - explicit TxDataHashMapAllocHandle(SliceAllocator *slice_allocator) - : slice_allocator_(slice_allocator) - {} - - TxDataHashMapAllocHandle(const TxDataHashMapAllocHandle &rhs) - { - slice_allocator_ = rhs.slice_allocator_; - } - - // do nothing - ObTxData *alloc_value() { return nullptr; } - // do nothing - void free_value(ObTxData *tx_data) { UNUSED(tx_data); } - - // construct TxDataHashNode with the memory allocated by slice allocator - TxDataHashNode *alloc_node(ObTxData *tx_data); - - // the memory allocated by slice allocator in tx data table is freed in link hash map - void free_node(TxDataHashNode *node); - void set_slice_allocator(SliceAllocator *slice_allocator) { slice_allocator_ = slice_allocator; } - -private: - void free_undo_list_(ObUndoStatusNode *node_ptr); - -private: - SliceAllocator *slice_allocator_; + int64_t flag_; }; class ObTxDataGuard { public: - ObTxDataGuard() : is_inited_(false), tx_data_(nullptr), tx_data_map_(nullptr) {} - virtual ~ObTxDataGuard() { reset(); } + ObTxDataGuard() : tx_data_(nullptr) {} + ~ObTxDataGuard() { reset(); } + ObTxDataGuard &operator=(ObTxDataGuard &rhs) = delete; + ObTxDataGuard(const ObTxDataGuard &other) = delete; - int init(ObTxData *tx_data, TxDataMap *tx_data_map) + int init(ObTxData *tx_data) { int ret = OB_SUCCESS; - if (IS_INIT) { - reset(); - } - if (OB_ISNULL(tx_data) || OB_ISNULL(tx_data_map)) { + reset(); + if (OB_ISNULL(tx_data)) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "init ObTxDataGuard with invalid arguments", KR(ret)); + } else if (tx_data->inc_ref() <= 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected ref cnt on tx data", KR(ret), KP(tx_data), KPC(tx_data)); + ob_abort(); } else { tx_data_ = tx_data; - tx_data_map_ = tx_data_map; - is_inited_ = true; } - return ret; } void reset() { - tx_data_map_->revert(tx_data_); - tx_data_ = nullptr; - tx_data_map_ = nullptr; - is_inited_ = false; + if (OB_NOT_NULL(tx_data_)) { + tx_data_->dec_ref(); + tx_data_ = nullptr; + } } - ObTxData &tx_data() { return *tx_data_; } + ObTxData *tx_data() { return tx_data_; } + const ObTxData *tx_data() const { return tx_data_; } + + TO_STRING_KV(KPC_(tx_data)); +public: + // void TEST_reset() + // { + // tx_data_ = nullptr; + // } private: - bool is_inited_; ObTxData *tx_data_; - TxDataMap *tx_data_map_; }; class ObTxDataMemtableWriteGuard diff --git a/src/storage/tx_storage/ob_access_service.cpp b/src/storage/tx_storage/ob_access_service.cpp index 0f55aee20..87a623dca 100644 --- a/src/storage/tx_storage/ob_access_service.cpp +++ b/src/storage/tx_storage/ob_access_service.cpp @@ -110,6 +110,8 @@ int ObAccessService::pre_check_lock( transaction::ObTransVersion::INVALID_TRANS_VERSION; transaction::ObTxReadSnapshot snapshot; snapshot.init_none_read(); + concurrent_control::ObWriteFlag write_flag; + write_flag.set_is_table_lock(); if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("ob access service is not running.", K(ret)); @@ -122,6 +124,7 @@ int ObAccessService::pre_check_lock( param.expired_time_, /*timeout*/ tx_desc, snapshot, + write_flag, ctx_guard))) { LOG_WARN("fail to check query allowed", K(ret), K(ls_id)); } else if (OB_ISNULL(ls = ctx_guard.get_ls_handle().get_ls())) { @@ -143,6 +146,9 @@ int ObAccessService::lock_obj( ObLS *ls = nullptr; transaction::ObTxReadSnapshot snapshot; snapshot.init_none_read(); + concurrent_control::ObWriteFlag write_flag; + write_flag.set_is_table_lock(); + if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("ob access service is not running.", K(ret)); @@ -155,6 +161,7 @@ int ObAccessService::lock_obj( param.expired_time_, /*timeout*/ tx_desc, snapshot, + write_flag, ctx_guard))) { LOG_WARN("fail to check query allowed", K(ret), K(ls_id)); } else if (OB_ISNULL(ls = ctx_guard.get_ls_handle().get_ls())) { @@ -177,6 +184,9 @@ int ObAccessService::unlock_obj( int64_t user_specified_snapshot = transaction::ObTransVersion::INVALID_TRANS_VERSION; transaction::ObTxReadSnapshot snapshot; snapshot.init_none_read(); + concurrent_control::ObWriteFlag write_flag; + write_flag.set_is_table_lock(); + if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("ob access service is not running.", K(ret)); @@ -189,6 +199,7 @@ int ObAccessService::unlock_obj( param.expired_time_, /*timeout*/ tx_desc, snapshot, + write_flag, ctx_guard))) { LOG_WARN("fail to check query allowed", K(ret), K(ls_id)); } else if (OB_ISNULL(ls = ctx_guard.get_ls_handle().get_ls())) { @@ -339,6 +350,7 @@ int ObAccessService::get_write_store_ctx_guard_( const int64_t timeout, transaction::ObTxDesc &tx_desc, const transaction::ObTxReadSnapshot &snapshot, + const concurrent_control::ObWriteFlag write_flag, ObStoreCtxGuard &ctx_guard) { int ret = OB_SUCCESS; @@ -355,7 +367,7 @@ int ObAccessService::get_write_store_ctx_guard_( auto &ctx = ctx_guard.get_store_ctx(); ctx.ls_ = ls; ctx.timeout_ = timeout; - if (OB_FAIL(ls->get_write_store_ctx(tx_desc, snapshot, ctx))) { + if (OB_FAIL(ls->get_write_store_ctx(tx_desc, snapshot, write_flag, ctx))) { LOG_WARN("can not get write store ctx", K(ret), K(ls_id), K(snapshot), K(tx_desc)); } } @@ -497,6 +509,7 @@ int ObAccessService::check_write_allowed_( dml_param.timeout_, tx_desc, dml_param.snapshot_, + dml_param.write_flag_, ctx_guard))) { LOG_WARN("get write store ctx failed", K(ret), K(ls_id), K(dml_param), K(tx_desc)); } else if (FALSE_IT(ctx_guard.get_store_ctx().tablet_id_ = tablet_id)) { diff --git a/src/storage/tx_storage/ob_access_service.h b/src/storage/tx_storage/ob_access_service.h index c819ba656..a6d98cc78 100644 --- a/src/storage/tx_storage/ob_access_service.h +++ b/src/storage/tx_storage/ob_access_service.h @@ -204,6 +204,7 @@ protected: const int64_t timeout, transaction::ObTxDesc &tx_desc, const transaction::ObTxReadSnapshot &snapshot, + const concurrent_control::ObWriteFlag write_flag, ObStoreCtxGuard &ctx_guard); int check_read_allowed_( const share::ObLSID &ls_id, diff --git a/src/storage/tx_storage/ob_tenant_freezer.cpp b/src/storage/tx_storage/ob_tenant_freezer.cpp old mode 100644 new mode 100755 index 59517d02a..e4ff10b11 --- a/src/storage/tx_storage/ob_tenant_freezer.cpp +++ b/src/storage/tx_storage/ob_tenant_freezer.cpp @@ -252,12 +252,41 @@ int ObTenantFreezer::tenant_freeze_() return ret; } +int ObTenantFreezer::ls_freeze(const share::ObLSID &ls_id) +{ + int ret = OB_SUCCESS; + ObLSService *ls_srv = MTL(ObLSService *); + ObLSHandle handle; + ObLS *ls = nullptr; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("[TenantFreezer] tenant freezer not inited", KR(ret)); + } else if (OB_FAIL(ls_srv->get_ls(ls_id, handle, ObLSGetMod::TXSTORAGE_MOD))) { + LOG_WARN("[TenantFreezer] fail to get ls", K(ret), K(ls_id)); + } else if (OB_ISNULL(ls = handle.get_ls())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("[TenantFreezer] ls is null", KR(ret), K(ls_id)); + } else if (OB_FAIL(ls->logstream_freeze())) { + LOG_WARN("[TenantFreezer] logstream freeze failed", KR(ret), K(ls_id)); + } + + return ret; +} + int ObTenantFreezer::tablet_freeze(const common::ObTabletID &tablet_id, const bool is_force_freeze, const bool is_sync) +{ + return tablet_freeze(ObLSID(ObLSID::INVALID_LS_ID), tablet_id, is_force_freeze, is_sync); +} + +int ObTenantFreezer::tablet_freeze(share::ObLSID ls_id, + const common::ObTabletID &tablet_id, + const bool is_force_freeze, + const bool is_sync) { int ret = OB_SUCCESS; - share::ObLSID ls_id; bool is_cache_hit = false; ObLSService *ls_srv = MTL(ObLSService *); ObLSHandle handle; @@ -267,15 +296,14 @@ int ObTenantFreezer::tablet_freeze(const common::ObTabletID &tablet_id, if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("[TenantFreezer] tenant freezer not inited", KR(ret)); - } else if (OB_UNLIKELY(nullptr == GCTX.location_service_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("[TenantFreezer] location service ptr is null", KR(ret)); - } else if (OB_FAIL(GCTX.location_service_->get(tenant_info_.tenant_id_, - tablet_id, - INT64_MAX, - is_cache_hit, - ls_id))) { - LOG_WARN("[TenantFreezer] fail to get ls id according to tablet_id", KR(ret), K(tablet_id)); + } else if (!ls_id.is_valid()) { + // if ls_id is invalid, get ls id by tablet id + if (OB_FAIL(GCTX.location_service_->get(tenant_info_.tenant_id_, tablet_id, INT64_MAX, is_cache_hit, ls_id))) { + LOG_WARN("[TenantFreezer] fail to get ls id according to tablet_id", KR(ret), K(tablet_id)); + } + } + + if (OB_FAIL(ret)) { } else if (OB_FAIL(ls_srv->get_ls(ls_id, handle, ObLSGetMod::TXSTORAGE_MOD))) { LOG_WARN("[TenantFreezer] fail to get ls", K(ret), K(ls_id)); } else if (OB_ISNULL(ls = handle.get_ls())) { diff --git a/src/storage/tx_storage/ob_tenant_freezer.h b/src/storage/tx_storage/ob_tenant_freezer.h old mode 100644 new mode 100755 index 24c984c55..cffded1b3 --- a/src/storage/tx_storage/ob_tenant_freezer.h +++ b/src/storage/tx_storage/ob_tenant_freezer.h @@ -57,10 +57,19 @@ public: int stop(); void wait(); + // freeze all the ls of this tenant. + // return the first failed code. + int tenant_freeze(); + + int ls_freeze(const share::ObLSID &ls_id); // freeze a tablet int tablet_freeze(const common::ObTabletID &tablet_id, const bool is_force_freeze = false, const bool is_sync = false); + int tablet_freeze(share::ObLSID ls_id, + const common::ObTabletID &tablet_id, + const bool is_force_freeze = false, + const bool is_sync = false); // check if this tenant's memstore is out of range, and trigger minor/major freeze. int check_and_do_freeze(); diff --git a/src/storage/tx_table/ob_tx_data_hash_map.cpp b/src/storage/tx_table/ob_tx_data_hash_map.cpp new file mode 100644 index 000000000..e3adda793 --- /dev/null +++ b/src/storage/tx_table/ob_tx_data_hash_map.cpp @@ -0,0 +1,145 @@ +/** + * 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_table/ob_tx_data_hash_map.h" +#include "storage/tx/ob_tx_data_define.h" +#include "lib/utility/ob_macro_utils.h" + +namespace oceanbase { +namespace storage { + +void ObTxDataHashMap::destroy() +{ + ObTxData *curr = nullptr; + ObTxData *next = nullptr; + for (int64_t i = 0; i < BUCKETS_CNT; ++i) { + curr = buckets_[i].next_; + buckets_[i].next_ = nullptr; + + while (OB_NOT_NULL(curr)) { + next = curr->hash_node_.next_; + curr->dec_ref(); + curr = next; + } + } + ob_free(buckets_); +} + +int ObTxDataHashMap::init() +{ + int ret = OB_SUCCESS; + void *ptr = ob_malloc(BUCKETS_CNT * sizeof(ObTxDataHashHeader)); + if (OB_ISNULL(ptr)) { + STORAGE_LOG(WARN, "allocate memory failed when init tx data hash map", K(ret)); + } else { + buckets_ = new (ptr) ObTxDataHashHeader[BUCKETS_CNT]; + for (int i = 0; i < BUCKETS_CNT; i++) { + buckets_[i].reset(); + } + } + return ret; +} + +int ObTxDataHashMap::insert(const transaction::ObTransID &key, ObTxData *value) +{ + int ret = OB_SUCCESS; + + if (!key.is_valid() || OB_ISNULL(value)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", K(key), KP(value)); + } else { + int64_t pos = get_pos(key); + + // atomic insert this value + while (true) { + ObTxData *next_value = ATOMIC_LOAD(&buckets_[pos].next_); + value->hash_node_.next_ = next_value; + if (next_value == ATOMIC_CAS(&buckets_[pos].next_, next_value, value)) { + if (value->inc_ref() <= 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected ref cnt on tx data", KR(ret), KPC(value)); + ob_abort(); + } + ATOMIC_INC(&total_cnt_); + break; + } + } + } + return ret; +} + +int ObTxDataHashMap::get(const transaction::ObTransID &key, ObTxDataGuard &guard) +{ + int ret = OB_SUCCESS; + ObTxData *value = nullptr; + + if (!key.is_valid()) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", K(key)); + } else { + const int64_t pos = get_pos(key); + ObTxData *cache_val = buckets_[pos].hot_cache_val_; + if (OB_NOT_NULL(cache_val) && cache_val->contain(key)) { + value = cache_val; + } else { + ObTxData *tmp_value = nullptr; + tmp_value = buckets_[pos].next_; + while (OB_NOT_NULL(tmp_value)) { + if (tmp_value->contain(key)) { + value = tmp_value; + buckets_[pos].hot_cache_val_ = value; + break; + } else { + tmp_value = tmp_value->hash_node_.next_; + } + } + } + } + + if (OB_ISNULL(value)) { + ret = OB_ENTRY_NOT_EXIST; + } else if (OB_FAIL(guard.init(value))) { + STORAGE_LOG(WARN, "get tx data from tx data hash map failed", KR(ret), KP(this), KPC(value)); + } + return ret; +} + +int ObTxDataHashMap::Iterator::get_next(ObTxDataGuard &guard) +{ + int ret = OB_SUCCESS; + ObTxData *next_val = nullptr; + + while (OB_SUCC(ret) && OB_ISNULL(next_val)) { + if (OB_NOT_NULL(val_)) { + next_val = val_; + val_ = val_->hash_node_.next_; + } else if (bucket_idx_ >= tx_data_map_.BUCKETS_CNT) { + ret = OB_ITER_END; + } else { + while (++bucket_idx_ < tx_data_map_.BUCKETS_CNT) { + val_ = tx_data_map_.buckets_[bucket_idx_].next_; + + if (OB_NOT_NULL(val_)) { + break; + } + } + } + } + + if (OB_SUCC(ret) && OB_NOT_NULL(next_val) && OB_FAIL(guard.init(next_val))) { + STORAGE_LOG(WARN, "init tx data guard failed when get next from iterator", KR(ret), KPC(next_val)); + } + return ret; +} + +} // namespace storage +} // namespace oceanbase \ No newline at end of file diff --git a/src/storage/tx_table/ob_tx_data_hash_map.h b/src/storage/tx_table/ob_tx_data_hash_map.h new file mode 100644 index 000000000..c0e42a7e9 --- /dev/null +++ b/src/storage/tx_table/ob_tx_data_hash_map.h @@ -0,0 +1,113 @@ + +/** + * 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_STORAGE_OB_TX_DATA_HASHMAP_ +#define OCEANBASE_STORAGE_OB_TX_DATA_HASHMAP_ + +#include "lib/ob_define.h" +#include "lib/utility/ob_print_utils.h" +#include "lib/container/ob_se_array.h" +#include "lib/allocator/ob_slice_alloc.h" +#include "storage/tx/ob_trans_define.h" + +namespace oceanbase { + +namespace transaction { +class ObTransID; +} + +namespace storage { +class ObTxData; +class ObTxDataGuard; + +class ObTxDataHashMap { +private: + static const int64_t MAX_CONCURRENCY = 32; + static const int64_t MAX_CONCURRENCY_MASK = MAX_CONCURRENCY - 1; + +public: + ObTxDataHashMap(const uint64_t buckets_cnt) + : BUCKETS_CNT(buckets_cnt), + BUCKETS_MOD_MASK(buckets_cnt - 1), + buckets_(nullptr), + total_cnt_(0) + {} + ~ObTxDataHashMap() + { + destroy(); + } + + int init(); + virtual void destroy(); + + int insert(const transaction::ObTransID &key, ObTxData *value); + int get(const transaction::ObTransID &key, ObTxDataGuard &guard); + + OB_INLINE int64_t get_pos(const transaction::ObTransID key) + { + return key.hash() & BUCKETS_MOD_MASK; + } + + OB_INLINE int64_t get_buckets_cnt() + { + return BUCKETS_CNT; + } + + OB_INLINE int64_t count() const + { + return ATOMIC_LOAD(&total_cnt_); + } + +private: + struct ObTxDataHashHeader { + ObTxData *next_; + ObTxData *hot_cache_val_; + + ObTxDataHashHeader() : next_(nullptr), hot_cache_val_(nullptr) {} + ~ObTxDataHashHeader() + { + destroy(); + } + void reset() + { + next_ = nullptr; + hot_cache_val_ = nullptr; + } + void destroy() { reset(); } + }; + +private: + const int64_t BUCKETS_CNT; + const int64_t BUCKETS_MOD_MASK; + ObTxDataHashHeader *buckets_; + int64_t total_cnt_; + +public: + class Iterator { + public: + Iterator(ObTxDataHashMap &tx_data_map) + : bucket_idx_(-1), val_(nullptr), tx_data_map_(tx_data_map) + {} + + int get_next(ObTxDataGuard &next_val); + + public: + int64_t bucket_idx_; + ObTxData *val_; + ObTxDataHashMap &tx_data_map_; + }; +}; + +} // namespace storage +} // namespace oceanbase +#endif // OCEANBASE_STORAGE_OB_TX_DATA_HASHMAP_ diff --git a/src/storage/tx_table/ob_tx_data_memtable.cpp b/src/storage/tx_table/ob_tx_data_memtable.cpp index 1aa65c408..21b774f07 100644 --- a/src/storage/tx_table/ob_tx_data_memtable.cpp +++ b/src/storage/tx_table/ob_tx_data_memtable.cpp @@ -11,119 +11,138 @@ */ #include "storage/tx_table/ob_tx_data_memtable.h" + #include "storage/compaction/ob_schedule_dag_func.h" #include "storage/compaction/ob_tablet_merge_task.h" #include "storage/ls/ob_freezer.h" +#include "storage/ls/ob_ls_tablet_service.h" +#include "storage/tablet/ob_tablet.h" #include "storage/tx/ob_tx_data_functor.h" #include "storage/tx_table/ob_tx_data_memtable_mgr.h" #include "storage/tx_table/ob_tx_data_table.h" #include "storage/tx_table/ob_tx_table_iterator.h" +#include "storage/tablet/ob_tablet.h" namespace oceanbase { + +using namespace transaction; +using namespace share; + namespace storage { -using namespace oceanbase::transaction; -using namespace oceanbase::share; + +int64_t ObTxDataMemtable::PERIODICAL_SELECT_INTERVAL_NS = 1000LL * 1000LL * 1000LL; int ObTxDataMemtable::init(const ObITable::TableKey &table_key, SliceAllocator *slice_allocator, ObTxDataMemtableMgr *memtable_mgr) { int ret = OB_SUCCESS; + if (IS_INIT) { ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "init tx data memtable twice", KR(ret)); + STORAGE_LOG(WARN, "init tx data memtable twice", KR(ret), K(table_key), KPC(memtable_mgr)); } else if (OB_ISNULL(slice_allocator) || OB_ISNULL(memtable_mgr)) { ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(WARN, "the slice_allocator or arena_allocator is nullptr", KR(ret)); + STORAGE_LOG(WARN, "the slice_allocator is nullptr", KR(ret), K(table_key), KPC(memtable_mgr)); } else if (OB_FAIL(ObITable::init(table_key))) { - STORAGE_LOG(WARN, "ObITable::init fail"); + STORAGE_LOG(WARN, "ObITable::init fail", KR(ret), K(table_key), KPC(memtable_mgr)); + } else if (FALSE_IT(init_arena_allocator_())) { + } else if (OB_FAIL(init_tx_data_map_())) { + STORAGE_LOG(WARN, "init tx data map failed.", KR(ret), K(table_key), KPC(memtable_mgr)); + } else if (OB_FAIL(buf_.reserve(common::OB_MAX_VARCHAR_LENGTH))) { + STORAGE_LOG(WARN, "reserve space for tx data memtable failed.", KR(ret), K(table_key), KPC(memtable_mgr)); } else { - is_iterating_ = false; - min_tx_scn_.set_max(); + for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { + min_tx_scn_[i] = SCN::max_scn(); + min_start_scn_[i] = SCN::max_scn(); + occupied_size_[i] = 0; + } + construct_list_done_ = false; + pre_process_done_ = false; max_tx_scn_.set_min(); - min_start_scn_.set_max(); inserted_cnt_ = 0; deleted_cnt_ = 0; write_ref_ = 0; - occupied_size_ = 0; last_insert_ts_ = 0; state_ = ObTxDataMemtable::State::ACTIVE; sort_list_head_.reset(); slice_allocator_ = slice_allocator; memtable_mgr_ = memtable_mgr; + row_key_array_.reuse(); - TxDataHashMapAllocHandle tx_data_alloc_handle(slice_allocator_); - ObMemAttr attr; - attr.tenant_id_ = MTL_ID(); - attr.label_ = "TX_DATA_TABLE"; - attr.ctx_id_ = ObCtxIds::DEFAULT_CTX_ID; - void *data_map_ptr = ob_malloc(sizeof(*tx_data_map_), attr); - if (OB_ISNULL(data_map_ptr)) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "allocate memory of tx_data_map_ failed", KR(ret)); - } else { - tx_data_map_ = new (data_map_ptr) TxDataMap(tx_data_alloc_handle, 64 << 10, 2 << 20); - if (OB_FAIL(tx_data_map_->init(attr.label_, attr.tenant_id_))) { - STORAGE_LOG(WARN, "tx_data_map_ init failed", KR(ret)); - } else { - is_inited_ = true; - } - } + DEBUG_iter_commit_ts_cnt_ = 0; + DEBUG_last_start_scn_ = SCN::min_scn(); + is_inited_ = true; } return ret; } +int ObTxDataMemtable::init_tx_data_map_() +{ + int ret = OB_SUCCESS; + + void *data_map_ptr = arena_allocator_.alloc(sizeof(*tx_data_map_)); + if (OB_ISNULL(data_map_ptr)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "allocate memory of tx_data_map_ failed", KR(ret)); + } else { + tx_data_map_ = new (data_map_ptr) TxDataMap(1 << 20/*2097152*/); + if (OB_FAIL(tx_data_map_->init())) { + STORAGE_LOG(WARN, "tx_data_map_ init failed", KR(ret)); + } else { + is_inited_ = true; + } + } + return ret; +} + +void ObTxDataMemtable::init_arena_allocator_() +{ + ObMemAttr attr; + attr.tenant_id_ = MTL_ID(); + attr.label_ = "TX_DATA_TABLE"; + attr.ctx_id_ = ObCtxIds::DEFAULT_CTX_ID; + arena_allocator_.set_attr(attr); +} + void ObTxDataMemtable::reset() { + for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { + min_tx_scn_[i] = SCN::max_scn(); + min_start_scn_[i] = SCN::max_scn(); + occupied_size_[i] = 0; + } + construct_list_done_ = false; + pre_process_done_ = false; key_.reset(); - is_iterating_ = false; - has_constructed_list_ = false; - min_tx_scn_.set_max(); max_tx_scn_.set_min(); - min_start_scn_.set_max(); inserted_cnt_ = 0; deleted_cnt_ = 0; write_ref_ = 0; - occupied_size_ = 0; last_insert_ts_ = 0; state_ = ObTxDataMemtable::State::INVALID; sort_list_head_.reset(); - reset_thread_local_list_(); if (OB_NOT_NULL(tx_data_map_)) { tx_data_map_->destroy(); - ob_free(tx_data_map_); + arena_allocator_.free(tx_data_map_); tx_data_map_ = nullptr; } slice_allocator_ = nullptr; memtable_mgr_ = nullptr; + buf_.reset(); + arena_allocator_.reset(); + row_key_array_.reuse(); freezer_ = nullptr; ObITable::reset(); + DEBUG_iter_commit_ts_cnt_ = 0; + DEBUG_last_start_scn_ = SCN::min_scn(); is_inited_ = false; } -void ObTxDataMemtable::reset_thread_local_list_() -{ - for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { - ObTxDataSortListNode *cur_node = local_sort_list_head_[i].next_; - while (OB_NOT_NULL(cur_node)) { - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); - cur_node = cur_node->next_; - if (false == tx_data->is_in_tx_data_table_) { - if (OB_ISNULL(tx_data_map_)) { - STORAGE_LOG(ERROR, "tx_data_map is unexpected nullptr", KP(tx_data_map_), KPC(tx_data)); - } else { - tx_data_map_->revert(tx_data); - } - } - } - local_sort_list_head_[i].reset(); - } -} - int ObTxDataMemtable::insert(ObTxData *tx_data) { common::ObTimeGuard tg("tx_data_memtable::insert", 100 * 1000); @@ -131,56 +150,29 @@ int ObTxDataMemtable::insert(ObTxData *tx_data) if (IS_NOT_INIT) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "tx data memtable is not init"); + } else if (OB_UNLIKELY(ObTxDataMemtable::State::FROZEN <= get_state())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, + "trying to insert a tx data into a tx data memtable in frozen/dumped state.", + KR(ret), + KP(this), + KPC(this)); } else if (OB_ISNULL(tx_data)) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "tx data is nullptr", KR(ret)); } else if (OB_ISNULL(tx_data_map_)) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "unexpected null value of tx_data_map_", KR(ret)); - } else if (true == tx_data->is_in_tx_data_table_) { - STORAGE_LOG(ERROR, "insert an exist tx data", KP(tx_data), KPC(tx_data), KR(ret), - KP(tx_data_map_)); - } else if (OB_FAIL(tx_data_map_->insert_and_get(tx_data->tx_id_, tx_data))) { - // insert_and_get fail - if (ret == OB_ENTRY_EXIST) { - STORAGE_LOG(ERROR, "insert an exist tx data", KP(tx_data), KPC(tx_data), KR(ret), - KP(tx_data_map_)); - } else { - STORAGE_LOG(ERROR, "insert the tx data into tx_data_map_ fail.", KP(tx_data), KPC(tx_data), + } else if (OB_FAIL(tx_data_map_->insert(tx_data->tx_id_, tx_data))) { + STORAGE_LOG(ERROR, "insert the tx data into tx_data_map_ fail.", KP(tx_data), KPC(tx_data), KR(ret), KP(tx_data_map_)); - } } else { tg.click(); // insert_and_get success - tx_data->is_in_tx_data_table_ = true; max_tx_scn_.inc_update(tx_data->end_scn_); - min_tx_scn_.dec_update(tx_data->end_scn_); - min_start_scn_.dec_update(tx_data->start_scn_); + atomic_update_(tx_data); ATOMIC_INC(&inserted_cnt_); tg.click(); - - int thread_idx = ::get_itid() % MAX_TX_DATA_TABLE_CONCURRENCY; - ObTxDataSortListNode *cur_node = ObTxData::get_sort_list_node_by_tx_data(tx_data); - tg.click(); - while (true) { - ObTxDataSortListNode *last_node = ATOMIC_LOAD(&local_sort_list_head_[thread_idx].next_); - cur_node->next_ = last_node; - if (last_node == ATOMIC_CAS(&local_sort_list_head_[thread_idx].next_, last_node, cur_node)) { - break; - } - } - tg.click(); - - // Note : a tx data may be deleted from memtable in ObTxDataTable::insert_into_memtable_ but the - // occupied_size would not be reduced because the memory will not be freed until freeze done. - int64_t tx_data_size = TX_DATA_SLICE_SIZE * (1LL + tx_data->undo_status_list_.undo_node_cnt_); - ATOMIC_FAA(&occupied_size_, tx_data_size); - - // TODO : @gengli remove this after tx data memtable flush stable - common::inc_update(&last_insert_ts_, ObTimeUtil::current_time_ns()); - - tx_data_map_->revert(tx_data); - tg.click(); } if (tg.get_diff() > 100000) { STORAGE_LOG(INFO, "tx data memtable insert cost too much time", K(tg)); @@ -189,35 +181,61 @@ int ObTxDataMemtable::insert(ObTxData *tx_data) return ret; } +void ObTxDataMemtable::atomic_update_(ObTxData *tx_data) +{ + int64_t thread_idx = ::get_itid() & MAX_CONCURRENCY_MOD_MASK; + min_tx_scn_[thread_idx].dec_update(tx_data->end_scn_); + min_start_scn_[thread_idx].dec_update(tx_data->start_scn_); + int64_t tx_data_size = TX_DATA_SLICE_SIZE * (1LL + tx_data->undo_status_list_.undo_node_cnt_); + ATOMIC_FAA(&occupied_size_[thread_idx], tx_data_size); +} + int ObTxDataMemtable::get_tx_data(const ObTransID &tx_id, ObTxDataGuard &tx_data_guard) { int ret = OB_SUCCESS; - ObTxData *tx_data = nullptr; if (IS_NOT_INIT) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "tx data memtable is not init"); - } else if (OB_FAIL(tx_data_map_->get(tx_id, tx_data))) { + } else if (OB_FAIL(tx_data_map_->get(tx_id, tx_data_guard))) { // This tx data is not in this tx data memtable - } else if (OB_FAIL(tx_data_guard.init(tx_data, tx_data_map_))) { - STORAGE_LOG(WARN, "init tx data guard fail.", KR(ret)); + if (OB_ENTRY_NOT_EXIST != ret) { + STORAGE_LOG(WARN, "get tx data from tx data map failed.", KR(ret), KPC(this)); + } } else { // get tx data success } return ret; } -int ObTxDataMemtable::get_tx_data(const transaction::ObTransID &tx_id, ObTxData *&tx_data) +int ObTxDataMemtable::pre_process_for_merge() { int ret = OB_SUCCESS; - tx_data = nullptr; - if (IS_NOT_INIT) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "tx data memtable is not init"); - } else if (OB_FAIL(tx_data_map_->get(tx_id, tx_data))) { - // This tx data is not in this tx data memtable + ObTxDataGuard fake_tx_data_guard; + fake_tx_data_guard.reset(); + bool inserted_fake = false; + + if (State::FROZEN != state_) { + // only do pre process for frozen tx data memtable + } else if (pre_process_done_) { + STORAGE_LOG(INFO, "call pre process more than once. skip pre process."); + } else if (OB_FAIL(memtable_mgr_->get_tx_data_table()->alloc_tx_data(fake_tx_data_guard))) { + STORAGE_LOG(WARN, "allocate tx data from tx data table failed.", KR(ret), KPC(this)); + } else if (OB_FAIL(prepare_tx_data_list())) { + STORAGE_LOG(WARN, "prepare tx data list failed.", KR(ret), KPC(this)); + } else if (OB_FAIL(do_sort_by_start_scn_())) { + STORAGE_LOG(WARN, "do sort by start log ts failed.", KR(ret), KPC(this)); + } else if (OB_FAIL(pre_process_commit_version_row_(fake_tx_data_guard.tx_data()))) { + STORAGE_LOG(WARN, "process commit version row failed.", KR(ret), KPC(this)); + } else if (OB_FAIL(insert_fake_tx_data_to_list_and_map_(fake_tx_data_guard.tx_data()))) { + STORAGE_LOG(WARN, "insert fake tx data to list and map failed.", KR(ret), KPC(this)); + } else if (OB_FAIL(do_sort_by_tx_id_())) { + STORAGE_LOG(WARN, "do sort by tx id failed.", KR(ret), KPC(this)); } else { - // get tx data success + STORAGE_LOG(INFO, "prepare for parallel merge done.", KR(ret), K(fake_tx_data_guard)); + pre_process_done_ = true; } + + STORAGE_LOG(INFO, "prepare for parallel merge done", KR(ret), K(fake_tx_data_guard)); return ret; } @@ -228,137 +246,472 @@ int ObTxDataMemtable::prepare_tx_data_list() if (ObTxDataMemtable::State::FROZEN != state_) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "Trying to dump a non-frozen tx data memtable.", KR(ret), KP(this)); - } else if (!can_iterate()) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG( - ERROR, - "trying to construct a tx data memtable scan iterator while another iterator is existed.", - KR(ret), KPC(this)); + } else if (construct_list_done_) { + STORAGE_LOG(INFO, "construct list more than once. skip this time", KP(this)); + } else if (OB_FAIL(construct_list_for_sort_())) { + STORAGE_LOG(WARN, "construct list for sort failed.", KR(ret)); } else { - if (!has_constructed_list_) { - if (OB_FAIL(construct_list_for_sort_())) { - STORAGE_LOG(WARN, "construct list for sort failed.", KR(ret)); - } else { - has_constructed_list_ = true; - } - } else { - // construct scan iterator for this tx data memtable twice or more - STORAGE_LOG(INFO, "construct tx data memtable scan iterator more than once", KPC(this)); - } - - // sort list with merge sort - if (OB_FAIL(ret)) { - } else if (OB_FAIL(do_sort_by_tx_id_())) { - STORAGE_LOG(ERROR, "prepare dump fail when do sort", KR(ret), KPC(this)); - } + construct_list_done_ = true; } - if (OB_FAIL(ret)) { - reset_is_iterating(); - } return ret; } -// We need construct a list to hold the tx datas and sort them. There are already some thread local -// list exist. Here we just iterate them and concat them. -// -// It is not strange except one thing : There are some tx datas deleted in the link_hash_map because -// of the transaction rollback. But, it is difficult to delete them in thread local list at the same -// time deleted in link hash map. So we mark them by is_in_tx_data_table filed and revert it now to -// free its memory. +// We need construct a list to hold the tx datas and sort them. Here we iterate the tx_data_hash_map and concat them. int ObTxDataMemtable::construct_list_for_sort_() { int ret = OB_SUCCESS; - int64_t start_construct_ts = ObTimeUtil::current_time_ns(); + TxDataMap::Iterator iter(*tx_data_map_); - ObTxDataSortListNode *pre_node = &sort_list_head_; - ObTxDataSortListNode *cur_node = nullptr; - - int64_t sort_list_node_cnt = 0; - int64_t skip_list_node_cnt = 0; - for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { - cur_node = local_sort_list_head_[i].next_; - local_sort_list_head_[i].reset(); - while (OB_NOT_NULL(cur_node)) { - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); - - if (false == tx_data->is_in_tx_data_table_) { - cur_node = cur_node->next_; - // TODO : @gengli remove log info after stable - // STORAGE_LOG(INFO, "skip one tx data", KPC(tx_data), KP(this), K(freezer_->get_ls_id())); - skip_list_node_cnt++; - // revert must behind move pointer - tx_data_map_->revert(tx_data); - } else { - pre_node->next_ = cur_node; - pre_node = cur_node; - cur_node = cur_node->next_; - sort_list_node_cnt++; - } - - } + ObTxDataGuard tx_data_guard; + tx_data_guard.reset(); + ObTxDataLinkNode *cur_node = &sort_list_head_; + int64_t DEBUG_iter_cnt = 0; + while (OB_SUCC(iter.get_next(tx_data_guard))) { + cur_node->next_ = tx_data_guard.tx_data(); + cur_node = &(tx_data_guard.tx_data()->sort_list_node_); + DEBUG_iter_cnt++; } - pre_node->next_ = nullptr; - - if (start_construct_ts <= last_insert_ts_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "Tx data is inserted after flushing is running.", KR(ret), K(start_construct_ts), K(last_insert_ts_), KPC(this)); + if (OB_ITER_END == ret) { + cur_node->next_ = nullptr; + ret = OB_SUCCESS; } else { - bool node_cnt_correct = (skip_list_node_cnt == deleted_cnt_) - && (skip_list_node_cnt + sort_list_node_cnt == inserted_cnt_) - && (sort_list_node_cnt == tx_data_map_->count()) - && (inserted_cnt_ - deleted_cnt_ == tx_data_map_->count()); + STORAGE_LOG(WARN, "construct list for sort failed", KR(ret), KPC(this)); + } - if (!node_cnt_correct) { + return ret; +} + +// This function is called after sorting tx_data by start_log_ts and the following steps is +// executed: +// 1. Select (start_log_ts, commit_version) point per second and push them into an array. +// 2. Read (start_log_ts, commit_version) array from the latest tx data sstable. +// 3. Get the recycle_scn to filtrate the point which is not needed any more. +// 4. Merge the arrays above. This procedure should filtrate the points are not needed and keep the +// commit versions monotonically increasing. +// 5. Serialize the merged array into one sstable row. +int ObTxDataMemtable::pre_process_commit_version_row_(ObTxData *fake_tx_data) +{ + int ret = OB_SUCCESS; + + SCN recycle_scn = SCN::min_scn(); + ObCommitVersionsArray cur_commit_versions; + ObCommitVersionsArray past_commit_versions; + ObCommitVersionsArray merged_commit_versions; + + if (OB_FAIL(fill_in_cur_commit_versions_(cur_commit_versions)/*step 1*/)) { + STORAGE_LOG(WARN, "periodical select commit version failed.", KR(ret)); + } else if (OB_FAIL(get_past_commit_versions_(past_commit_versions)/*step 2*/)) { + STORAGE_LOG(WARN, "get past commit versions failed.", KR(ret)); + } else if (OB_FAIL(memtable_mgr_->get_tx_data_table()->get_recycle_scn(recycle_scn) /*step 3*/)) { + STORAGE_LOG(WARN, "get recycle ts failed.", KR(ret)); + } else if (OB_FAIL(merge_cur_and_past_commit_verisons_(recycle_scn, cur_commit_versions,/*step 4*/ + past_commit_versions, + merged_commit_versions))) { + STORAGE_LOG(WARN, "merge current and past commit versions failed.", KR(ret)); + } else if (!merged_commit_versions.is_valid()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, - "sort list count is not equal to inserted tx data count", + "invalid merged commit versions", KR(ret), - K(inserted_cnt_), - K(deleted_cnt_), - K(skip_list_node_cnt), - K(sort_list_node_cnt), - K(tx_data_map_->count()), - KPC(this)); + K(cur_commit_versions), + K(past_commit_versions), + K(merged_commit_versions)); + } else if (DEBUG_try_calc_upper_and_check_(merged_commit_versions)) { + STORAGE_LOG(ERROR, "check calc upper trans version failed", K(merged_commit_versions)); + } else { + int64_t pos = 0; + int64_t serialize_size = merged_commit_versions.get_serialize_size(); + + if (serialize_size > common::OB_MAX_VARCHAR_LENGTH) { + // TODO : @gengli multiple rows + int ret = OB_NOT_SUPPORTED; + STORAGE_LOG(WARN, "serialize data is too large", KR(ret), K(serialize_size)); + } else if (OB_FAIL(buf_.reserve(serialize_size))) { + STORAGE_LOG(WARN, "Failed to reserve local buffer", KR(ret), K(serialize_size), K(merged_commit_versions)); + } else if (OB_FAIL(merged_commit_versions.serialize(buf_.get_ptr(), serialize_size, pos))){ + STORAGE_LOG(WARN, "serialize merged commit versions failed", KR(ret), K(merged_commit_versions)); + } else { + // pre_processs commit version row done. + // Here we use commit_version_ and start_log_ts as two int64_t + fake_tx_data->tx_id_ = INT64_MAX; + fake_tx_data->commit_version_.convert_for_tx(serialize_size); + fake_tx_data->start_scn_.convert_for_tx((int64_t)buf_.get_ptr()); + fake_tx_data->undo_status_list_.head_ = nullptr; } } return ret; } -int ObTxDataMemtable::prepare_commit_scn_list() +int ObTxDataMemtable::fill_in_cur_commit_versions_(ObCommitVersionsArray &cur_commit_versions) +{ + int ret = OB_SUCCESS; + ObCommitVersionsArray::Node node; + ProcessCommitVersionData process_data(sort_list_head_.next_, SCN::min_scn(), SCN::min_scn()); + DEBUG_iter_commit_ts_cnt_ = 0; + DEBUG_last_start_scn_ = SCN::min_scn(); + + while (OB_SUCC(periodical_get_next_commit_version_(process_data, node))) { + cur_commit_versions.array_.push_back(node); + } + + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "Unexpected error occurs when periodical select commit version", KR(ret), + KPC(this)); + } + + return ret; +} + +int ObTxDataMemtable::periodical_get_next_commit_version_(ProcessCommitVersionData &process_data, + ObCommitVersionsArray::Node &node) +{ + int ret = OB_SUCCESS; + ObTxData *tx_data = nullptr; + SCN &cur_max_commit_version = process_data.cur_max_commit_version_; + SCN &pre_start_scn = process_data.pre_start_scn_; + + while (OB_SUCC(ret) && nullptr != process_data.cur_tx_data_) { + ObTxData *tmp_tx_data = process_data.cur_tx_data_; + process_data.cur_tx_data_ = process_data.cur_tx_data_->sort_list_node_.next_; + process_data.DEBUG_iter_commit_scn_cnt_++; + + // avoid rollback or abort transaction influencing commit versions array + if (ObTxData::COMMIT != tmp_tx_data->state_) { + continue; + } else { + tx_data = tmp_tx_data; + } + + if (process_data.DEBUG_last_start_scn_ > tx_data->start_scn_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected start log ts order", K(DEBUG_last_start_scn_), KPC(tx_data)); + break; + } else { + process_data.DEBUG_last_start_scn_ = tx_data->start_scn_; + } + + // update pre_commit_version + if (tx_data->commit_version_ > cur_max_commit_version) { + cur_max_commit_version = tx_data->commit_version_; + } + + // If this tx data is the first tx data in sorted list or its start_log_ts is 1_s larger than + // the pre_start_scn, we use this start_log_ts to calculate upper_trans_version + if (SCN::min_scn() == pre_start_scn || + tx_data->start_scn_ >= SCN::plus(pre_start_scn, PERIODICAL_SELECT_INTERVAL_NS/*1s*/)) { + pre_start_scn = tx_data->start_scn_; + break; + } + } + + if (nullptr != tx_data) { + node.start_scn_ = tx_data->start_scn_; + // use cur_max_commit_version_ to keep the commit versions monotonically increasing + node.commit_version_ = cur_max_commit_version; + tx_data = nullptr; + } else if (nullptr == process_data.cur_tx_data_) { + ret = OB_ITER_END; + } + + return ret; +} + +int ObTxDataMemtable::get_past_commit_versions_(ObCommitVersionsArray &past_commit_versions) +{ + int ret = OB_SUCCESS; + ObLSTabletService *tablet_svr = get_tx_data_memtable_mgr()->get_ls_tablet_svr(); + // Must copy iter param ! + ObTableIterParam iter_param = get_tx_data_memtable_mgr()->get_tx_data_table()->get_read_schema().iter_param_; + ObTabletHandle &tablet_handle = iter_param.tablet_handle_; + + if (tablet_handle.is_valid()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "tablet handle should be empty", KR(ret), K(tablet_handle)); + } else if (OB_ISNULL(tablet_svr)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "tablet svr is nullptr", KR(ret), KPC(this)); + } else if (OB_FAIL(tablet_svr->get_tablet(LS_TX_DATA_TABLET, tablet_handle))) { + STORAGE_LOG(WARN, "get tablet from ls tablet service failed.", KR(ret)); + } else if (OB_UNLIKELY(!tablet_handle.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "invalid tablet handle", KR(ret), K(tablet_handle)); + } else { + // get the lastest sstable + ObITable *table + = tablet_handle.get_obj()->get_table_store().get_minor_sstables().get_boundary_table( + true /*is_last*/); + + if (OB_NOT_NULL(table)) { + ObCommitVersionsGetter getter(iter_param, table); + if (OB_FAIL(getter.get_next_row(past_commit_versions))) { + STORAGE_LOG(WARN, "get commit versions from tx data sstable failed.", KR(ret)); + } + } else { + STORAGE_LOG(DEBUG, "There is no tx data sstable yet", KR(ret), KPC(table)); + } + } + + return ret; +} + +int ObTxDataMemtable::merge_cur_and_past_commit_verisons_(const SCN recycle_scn, + ObCommitVersionsArray &cur_commit_versions, + ObCommitVersionsArray &past_commit_versions, + ObCommitVersionsArray &merged_commit_versions) +{ + int ret = OB_SUCCESS; + ObIArray &cur_arr = cur_commit_versions.array_; + ObIArray &past_arr = past_commit_versions.array_; + ObIArray &merged_arr = merged_commit_versions.array_; + + int64_t cur_size = cur_commit_versions.get_serialize_size(); + int64_t past_size = past_commit_versions.get_serialize_size(); + int64_t step_len = 1; + if (cur_size + past_size > common::OB_MAX_VARCHAR_LENGTH) { + STORAGE_LOG(INFO, + "Too Much Pre-Process Data to Desirialize", + K(recycle_scn), + K(past_size), + K(cur_size), + "past_array_count", past_commit_versions.array_.count(), + "cur_array_count", cur_commit_versions.array_.count()); + step_len = step_len + ((cur_size + past_size) / OB_MAX_VARCHAR_LENGTH); + } + + // here we merge the past commit versions and current commit versions. To keep merged array correct, the node in past + // array whose start_scn is larger than the minimum start_scn in current array will be dropped. The reason is in this + // issue: https://work.aone.alibaba-inc.com/issue/43389863 + SCN cur_min_start_scn = cur_arr.count() > 0 ? cur_arr.at(0).start_scn_ : SCN::max_scn(); + SCN max_commit_version = SCN::min_scn(); + if (OB_FAIL(merge_pre_process_node_( + step_len, cur_min_start_scn, recycle_scn, past_arr, max_commit_version, merged_arr))) { + STORAGE_LOG(WARN, "merge past commit versions failed.", KR(ret), K(past_arr), KPC(this)); + } else if (OB_FAIL(merge_pre_process_node_( + step_len, SCN::max_scn() /*start_scn_limit*/, recycle_scn, cur_arr, max_commit_version, merged_arr))) { + STORAGE_LOG(WARN, "merge current commit versions failed.", KR(ret), K(cur_arr), KPC(this)); + } else if (0 == merged_arr.count()) { + if (OB_FAIL(merged_arr.push_back(ObCommitVersionsArray::Node(SCN::max_scn(), SCN::max_scn())))) { + STORAGE_LOG(WARN, "push back commit version node failed.", KR(ret), KPC(this)); + } else { + STORAGE_LOG(INFO, "push back an INT64_MAX node for upper trans version calculation", K(merged_arr)); + } + } + + STORAGE_LOG(INFO, + "genenrate commit versions array finish.", + K(recycle_scn), + K(step_len), + "past_array_count", past_commit_versions.array_.count(), + "cur_array_count", cur_commit_versions.array_.count(), + "merged_array_count", merged_commit_versions.array_.count()); + + return ret; +} + +int ObTxDataMemtable::merge_pre_process_node_(const int64_t step_len, + const SCN start_scn_limit, + const SCN recycle_scn, + const ObIArray &data_arr, + SCN &max_commit_version, + ObIArray &merged_arr) +{ + int ret = OB_SUCCESS; + int64_t arr_len = data_arr.count(); + if (arr_len <= 0) { + // skip push back + } else { + // push back pre-process node except the last one + int64_t i = 0; + for (; OB_SUCC(ret) && i < arr_len - 1; i += step_len) { + if (data_arr.at(i).start_scn_ >= start_scn_limit) { + break; + } + max_commit_version = std::max(max_commit_version, data_arr.at(i).commit_version_); + ObCommitVersionsArray::Node new_node(data_arr.at(i).start_scn_, max_commit_version); + if (new_node.commit_version_ <= recycle_scn) { + // this tx data should be recycled + // do nothing + } else if (OB_FAIL(merged_arr.push_back(new_node))) { + STORAGE_LOG(WARN, "push back commit version node failed.", KR(ret), KPC(this)); + } + } + + // push back the last pre-process node + max_commit_version = std::max(max_commit_version, data_arr.at(arr_len - 1).commit_version_); + if (OB_SUCC(ret) && data_arr.at(arr_len - 1).start_scn_ < start_scn_limit) { + ObCommitVersionsArray::Node new_node(data_arr.at(arr_len - 1).start_scn_, max_commit_version); + if (OB_FAIL(merged_arr.push_back(new_node))) { + STORAGE_LOG(WARN, "push back commit version node failed.", KR(ret), KPC(this)); + } + } + + } + return ret; +} + +int ObTxDataMemtable::insert_fake_tx_data_to_list_and_map_(ObTxData *fake_tx_data) { int ret = OB_SUCCESS; - if (ObTxDataMemtable::State::FROZEN != state_) { - if (ObTxDataMemtable::State::RELEASED == state_) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG( - WARN, "tx data memtable has been released. A concurrent logstream removing may happend", KR(ret), KP(this)); - } else { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "Trying to dump a non-frozen tx data memtable.", KR(ret), KP(this)); - } - } else if (OB_FAIL(do_sort_by_start_scn_())) { - STORAGE_LOG(ERROR, "prepare dump fail when do sort", KR(ret)); + // insert fake tx data into link hash map to release its memory after flushing + if (OB_FAIL(tx_data_map_->insert(fake_tx_data->tx_id_.get_id(), fake_tx_data))) { + STORAGE_LOG(WARN, "insert fake tx data into tx data map failed.", KR(ret), KPC(this)); + } else { + fake_tx_data->sort_list_node_.next_ = sort_list_head_.next_; + sort_list_head_.next_ = fake_tx_data; + ATOMIC_INC(&inserted_cnt_); } return ret; } +int ObTxDataMemtable::get_split_ranges(const ObStoreRowkey *start_key, + const ObStoreRowkey *end_key, + const int64_t part_cnt, + common::ObIArray &range_array) +{ + UNUSED(start_key); + UNUSED(end_key); + int ret = OB_SUCCESS; + + if (!pre_process_done_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "need do pre process before flushing tx data table", KR(ret), KPC(this)); + } else if (OB_FAIL(prepare_array_space_(part_cnt))) { + STORAGE_LOG(WARN, "prepare array space failed", KR(ret), KPC(this)); + } else if (FALSE_IT(row_key_array_.at(0).assign(0))) { + } else if (OB_FAIL(push_range_bounds_(part_cnt))) { + STORAGE_LOG(WARN, "push range bounds failed", KR(ret), KPC(this)); + } else if (FALSE_IT(row_key_array_.at(part_cnt).assign(INT64_MAX))) { + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt; i++) { + ObStoreRange merge_range; + merge_range.set_start_key(row_key_array_.at(i).get_rowkey()); + merge_range.set_end_key(row_key_array_.at(i + 1).get_rowkey()); + merge_range.set_left_open(); + merge_range.set_right_closed(); + if (OB_FAIL(range_array.push_back(merge_range))) { + STORAGE_LOG(WARN, "Failed to push back the merge range to array", KR(ret), K(merge_range)); + } + } + + } + + STORAGE_LOG(INFO, "generate range bounds for parallel dump tx data memtable:", K(row_key_array_), K(tx_id_2_cnt_)); + return ret; +} + +int ObTxDataMemtable::prepare_array_space_(const int64_t part_cnt) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(tx_id_2_cnt_.reserve(part_cnt + 1))) { + STORAGE_LOG(WARN, "reserve space for tx id to count array failed.", KR(ret), KPC(this)); + } else if (OB_FAIL(row_key_array_.reserve(part_cnt + 1))) { + STORAGE_LOG(WARN, "reserve space for fake row key array failed.", KR(ret), KPC(this)); + } else if (FALSE_IT(row_key_array_.reuse())) { + } else if (FALSE_IT(tx_id_2_cnt_.reuse())) { + } else { + for (int64_t i = 0; OB_SUCC(ret) && i <= part_cnt; i++) { + if (OB_FAIL(row_key_array_.push_back(TxDataFakeRowKey()))) { + STORAGE_LOG(WARN, "push back tx data fake row key failed", KR(ret), K(row_key_array_)); + } + } + } + return ret; +} + +int ObTxDataMemtable::push_range_bounds_(const int64_t part_cnt) +{ + int ret = OB_SUCCESS; + int64_t tx_data_count = get_tx_data_count(); + if (tx_data_count <= part_cnt) { + // do not need parallel merge. set some empty ranges + for (int64_t range_idx = 1; range_idx < part_cnt; range_idx++) { + row_key_array_.at(range_idx).assign(0); + } + } else { + int64_t data_cnt_in_one_range = tx_data_count / part_cnt; + int64_t pre_tx_id = 0; + int64_t last_tx_data_count = tx_data_count; + ObTxDataLinkNode *cur_node = &sort_list_head_; + + // iterate to find the bounds of the ranges + for (int64_t range_idx = 1; OB_SUCC(ret) && range_idx < part_cnt; range_idx++) { + + for (int64_t data_cnt = 0; data_cnt < data_cnt_in_one_range; data_cnt++) { + if (OB_NOT_NULL(cur_node->next_)) { + cur_node = &(cur_node->next_->sort_list_node_); + } else { + // this break should not be executed + break; + } + } + + // here we find the last node in a single range + ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); + const int64_t tx_id = tx_data->tx_id_.get_id(); + row_key_array_.at(range_idx).assign(tx_id); + + // push back tx data count to tx_id_2_cnt_ array + if (OB_FAIL(tx_id_2_cnt_.push_back(TxId2CntPair(transaction::ObTransID(pre_tx_id), data_cnt_in_one_range)))) { + STORAGE_LOG(WARN, + "push back tx id to count pair failed.", + KR(ret), + K(pre_tx_id), + K(data_cnt_in_one_range), + K(part_cnt), + KPC(this)); + } else { + pre_tx_id = tx_id; + last_tx_data_count = last_tx_data_count - data_cnt_in_one_range; + } + } + + tx_id_2_cnt_.push_back(TxId2CntPair(transaction::ObTransID(pre_tx_id), last_tx_data_count)); + } + + return ret; +} + +int ObTxDataMemtable::get_tx_data_cnt_by_tx_id(const transaction::ObTransID &tx_id, int64_t &tx_data_count) +{ + int ret = OB_SUCCESS; + tx_data_count = -1; + for (int i = 0; i < tx_id_2_cnt_.count(); i++) { + if (tx_id_2_cnt_.at(i).tx_id_ == tx_id) { + tx_data_count = tx_id_2_cnt_.at(i).tx_data_count_; + break; + } + } + + if (-1 == tx_data_count) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(ERROR, "this tx id is not in tx_id_2_cnt_ array.", KR(ret), K(tx_id), K(tx_id_2_cnt_)); + } + return ret; +} + int ObTxDataMemtable::scan(const ObTableIterParam ¶m, ObTableAccessContext &context, const blocksstable::ObDatumRange &range, ObStoreRowIterator *&row_iter) { int ret = OB_SUCCESS; - UNUSED(range); + ObTxDataMemtableScanIterator *scan_iter_ptr = nullptr; void *scan_iter_buff = nullptr; if (IS_NOT_INIT) { ret = OB_NOT_INIT; STORAGE_LOG(WARN, "ob tx data memtable is not inited.", KR(ret), KPC(this)); + } else if (!pre_process_done_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "need do pre process before flushing tx data table", KR(ret), KPC(this)); } else if (OB_UNLIKELY(!param.is_valid() || !context.is_valid())) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid param", KR(ret), K(param), K(context)); @@ -371,13 +724,14 @@ int ObTxDataMemtable::scan(const ObTableIterParam ¶m, STORAGE_LOG(WARN, "construct ObTxDataMemtableScanIterator fail", "scan_iter_buffer", scan_iter_buff, "scan_iter_ptr", scan_iter_ptr, KR(ret)); } else if (FALSE_IT(scan_iter_ptr = new (scan_iter_buff) ObTxDataMemtableScanIterator( - memtable_mgr_->get_tx_data_table()->get_read_schema().iter_param_))) { + memtable_mgr_->get_tx_data_table()->get_read_schema().iter_param_, range))) { } else if (OB_FAIL(scan_iter_ptr->init(this))) { STORAGE_LOG(WARN, "init scan_iter_ptr fail.", KR(ret)); } else { // tx data memtable scan iterator init success row_iter = scan_iter_ptr; } + return ret; } @@ -402,19 +756,16 @@ bool ObTxDataMemtable::can_be_minor_merged() return ready_for_flush(); } -bool ObTxDataMemtable::contain_tx_data(transaction::ObTransID tx_id) -{ - bool bool_ret = OB_ENTRY_EXIST == tx_data_map_->contains_key(tx_id); - return bool_ret; -} - int ObTxDataMemtable::remove(transaction::ObTransID tx_id) { - ATOMIC_INC(&deleted_cnt_); - // TODO : @gengli remove log info after stable - STORAGE_LOG(INFO, "remove one tx data", K(tx_id), KP(this), K(freezer_->get_ls_id())); + // ATOMIC_INC(&deleted_cnt_); + // // TODO : @gengli remove log info after stable + // STORAGE_LOG(INFO, "remove one tx data", K(tx_id), KP(this), K(freezer_->get_ls_id())); + // ObTxData *tx_data = NULL; + // tx_data_map_->get(tx_id, tx_data); - return tx_data_map_->del(tx_id); + // return tx_data_map_->del(tx_id, tx_data); + return 0; } bool ObTxDataMemtable::ready_for_flush() @@ -434,7 +785,7 @@ bool ObTxDataMemtable::ready_for_flush() STORAGE_LOG(WARN, "get_max_consequent_callbacked_scn failed", K(ret), K(freezer_->get_ls_id())); } else if (max_consequent_callbacked_scn >= key_.scn_range_.end_scn_) { state_ = ObTxDataMemtable::State::FROZEN; - set_snapshot_version(min_tx_scn_); + set_snapshot_version(get_min_tx_scn()); bool_ret = true; } else { const SCN &freeze_scn = key_.scn_range_.end_scn_; @@ -470,7 +821,7 @@ int ObTxDataMemtable::do_sort_by_tx_id_() int ret = OB_SUCCESS; // TODO : optimize merge sort with multiple threads merge_sort_(&get_tx_id_, sort_list_head_.next_); - // sort_list_head_.next_ = quick_sort_(&get_tx_id_, sort_list_head_.next_); + return ret; } @@ -484,126 +835,16 @@ int ObTxDataMemtable::do_sort_by_start_scn_() int ret = OB_SUCCESS; merge_sort_(&get_start_ts_, sort_list_head_.next_); // sort_list_head_.next_ = quick_sort_(&get_start_ts_, sort_list_head_.next_); + return ret; } -int ObTxDataMemtable::DEBUG_check_sort_result_(int64_t (*get_key)(const ObTxData &)) +void ObTxDataMemtable::merge_sort_(int64_t (*get_key)(const ObTxData &), ObTxData *&head) { - int ret = OB_SUCCESS; + ObTxData *left_list = nullptr; + ObTxData *right_list = nullptr; - ObTxDataSortListNode *cur_node = sort_list_head_.next_; - ObTxData *pre_tx_data = nullptr; - int64_t pre_key = -1; - - STORAGE_LOG(INFO, "start check sort result", KPC(this)); - - while (OB_SUCC(ret) && OB_NOT_NULL(cur_node)) { - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); - cur_node = cur_node->next_; - int64_t cur_key = get_key(*tx_data); - if (cur_key < pre_key) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "check sort result failed", KR(ret), KPC(pre_tx_data), KPC(tx_data)); - } else { - pre_key = cur_key; - pre_tx_data = tx_data; - } - } - - STORAGE_LOG(INFO, "finish check sort result", KR(ret), KPC(this)); - return ret; -} - -/* - * This quick sort based on linked list is a litte bit diffrent to normal quick sort. It uses the - * key of head node to split the whole list to a small_list and a large_list. Then it connects - * small_list, head_node and large_list. - * - * For example : First, we get a list like this: - * ┌──────┐ ┌──────┐ ┌──────┐ ┌──────┐ ┌──────┐ ┌──────┐ - * │ │ │ │ │ │ │ │ │ │ │ │ - * │ 4 ├───►│ 8 ├───►│ 6 ├───►│ 2 ├───►│ 7 ├───►│ 1 ├───►nullptr - * │ │ │ │ │ │ │ │ │ │ │ │ - * └──────┘ └──────┘ └──────┘ └──────┘ └──────┘ └──────┘ - * ▲ - * │ - * │ - * head - * - * It is splited by the first node (key = 4): - * ┌──────────────┐ ┌──────┐ ┌──────┐ - * │ │ │ │ │ │ - * │ small_head ├────►│ 2 ├───►│ 1 ├───►nullptr - * │ │ │ │ │ │ - * └──────────────┘ └──────┘ └──────┘ - * ▲ - * │ - * │ - * ┌──────┐ small_ptr - * │ │ - * │ 4 │────►nullptr - * │ │ - * └──────┘ - * ▲ - * │ - * │ - * head - * - * ┌──────────────┐ ┌──────┐ ┌──────┐ ┌──────┐ - * │ │ │ │ │ │ │ │ - * │ large_head ├────►│ 8 ├───►│ 6 ├───►│ 7 ├───►nullptr - * │ │ │ │ │ │ │ │ - * └──────────────┘ └──────┘ └──────┘ └──────┘ - * ▲ - * │ - * │ - * large_ptr - */ -ObTxDataSortListNode *ObTxDataMemtable::quick_sort_(int64_t (*get_key)(const ObTxData &), ObTxDataSortListNode *head) -{ - if (OB_ISNULL(head) || OB_ISNULL(head->next_)) { - return head; - } - - ObTxDataSortListNode small_head; - ObTxDataSortListNode large_head; - ObTxDataSortListNode *small_ptr = &small_head;; - ObTxDataSortListNode *large_ptr = &large_head;; - ObTxDataSortListNode *node_ptr = head->next_; - int64_t mid_key = get_key(*ObTxData::get_tx_data_by_sort_list_node(head)); - - while (OB_NOT_NULL(node_ptr)) { - ObTxDataSortListNode *cur_node = node_ptr; - node_ptr = node_ptr->next_; - int64_t cur_key = get_key(*ObTxData::get_tx_data_by_sort_list_node(cur_node)); - if (cur_key < mid_key) { - small_ptr->next_ = cur_node; - small_ptr = cur_node; - } else { - large_ptr->next_ = cur_node; - large_ptr = cur_node; - } - } - - small_ptr->next_ = head; - head->next_ = nullptr; - large_ptr->next_ = nullptr; - - small_head.next_ = quick_sort_(get_key, small_head.next_); - large_head.next_ = quick_sort_(get_key, large_head.next_); - - head->next_ = large_head.next_; - - return small_head.next_; -} - -void ObTxDataMemtable::merge_sort_(int64_t (*get_key)(const ObTxData &), ObTxDataSortListNode *&head) -{ - ObTxDataSortListNode *left_list = nullptr; - ObTxDataSortListNode *right_list = nullptr; - - - if (OB_ISNULL(head) || OB_ISNULL(head->next_)) { + if (OB_ISNULL(head) || OB_ISNULL(head->sort_list_node_.next_)) { return; } @@ -616,67 +857,65 @@ void ObTxDataMemtable::merge_sort_(int64_t (*get_key)(const ObTxData &), ObTxDat head = merge_sorted_list_(get_key, left_list, right_list); } -ObTxDataSortListNode *ObTxDataMemtable::merge_sorted_list_(int64_t (*get_key)(const ObTxData &), - ObTxDataSortListNode *left_list, - ObTxDataSortListNode *right_list) +ObTxData *ObTxDataMemtable::merge_sorted_list_(int64_t (*get_key)(const ObTxData &), + ObTxData *left_list, + ObTxData *right_list) { - ObTxDataSortListNode res; - ObTxDataSortListNode *insert_pos = &res; + ObTxData dummy_head; + ObTxData *insert_pos = &dummy_head; bool is_first_loop = true; ObTransID left_key = INT64_MAX; ObTransID right_key = INT64_MAX; while (nullptr != left_list && nullptr != right_list) { if (OB_UNLIKELY(is_first_loop)) { - left_key = get_key(*ObTxData::get_tx_data_by_sort_list_node(left_list)); - right_key = get_key(*ObTxData::get_tx_data_by_sort_list_node(right_list)); + left_key = get_key(*left_list); + right_key = get_key(*right_list); is_first_loop = false; } if (cmp_key_(left_key, right_key) < 0) { - insert_pos->next_ = left_list; - left_list = left_list->next_; + insert_pos->sort_list_node_.next_ = left_list; + left_list = left_list->sort_list_node_.next_; if (OB_NOT_NULL(left_list)) { - left_key = get_key(*ObTxData::get_tx_data_by_sort_list_node(left_list)); + left_key = get_key(*left_list); } } else { - insert_pos->next_ = right_list; - right_list = right_list->next_; + insert_pos->sort_list_node_.next_ = right_list; + right_list = right_list->sort_list_node_.next_; if (OB_NOT_NULL(right_list)) { - right_key = get_key(*ObTxData::get_tx_data_by_sort_list_node(right_list)); + right_key = get_key(*right_list); } } - insert_pos = insert_pos->next_; + insert_pos = insert_pos->sort_list_node_.next_; } if (OB_ISNULL(left_list)) { - insert_pos->next_ = right_list; + insert_pos->sort_list_node_.next_ = right_list; } else { - insert_pos->next_ = left_list; + insert_pos->sort_list_node_.next_ = left_list; } - return res.next_; + return dummy_head.sort_list_node_.next_; } -void ObTxDataMemtable::split_list_(ObTxDataSortListNode *head, - ObTxDataSortListNode *&left_list, - ObTxDataSortListNode *&right_list) +void ObTxDataMemtable::split_list_(ObTxData *head, ObTxData *&left_list, ObTxData *&right_list) { - ObTxDataSortListNode *slow = head; - ObTxDataSortListNode *fast = head->next_; + ObTxData *slow = head; + ObTxData *fast = head->sort_list_node_.next_; while (nullptr != fast) { - fast = fast->next_; + fast = fast->sort_list_node_.next_; if (nullptr != fast) { - slow = slow->next_; - fast = fast->next_; + slow = slow->sort_list_node_.next_; + fast = fast->sort_list_node_.next_; } } left_list = head; - right_list = slow->next_; - slow->next_ = nullptr; + right_list = slow->sort_list_node_.next_; + slow->sort_list_node_.next_ = nullptr; } int ObTxDataMemtable::cmp_key_(const int64_t &lhs, const int64_t &rhs) @@ -715,25 +954,25 @@ int ObTxDataMemtable::dump2text(const char *fname) auto tenant_id = MTL_ID(); fprintf(fd, "tenant_id=%ld ls_id=%ld\n", tenant_id, ls_id); fprintf(fd, - "memtable: key=%s is_inited=%d is_iterating=%d has_constructed_list=%d min_tx_log_scn=%s max_tx_log_scn=%s " - "min_start_log_scn=%s inserted_cnt=%ld deleted_cnt=%ld write_ref=%ld occupied_size=%ld last_insert_ts=%ld " + "memtable: key=%s is_inited=%d construct_list_done=%d pre_process_done=%d min_tx_log_ts=%s max_tx_log_ts=%s " + "min_start_log_ts=%s inserted_cnt=%ld deleted_cnt=%ld write_ref=%ld occupied_size=%ld last_insert_ts=%ld " "state=%d\n", S(key_), is_inited_, - is_iterating_, - has_constructed_list_, - to_cstring(min_tx_scn_), + construct_list_done_, + pre_process_done_, + to_cstring(get_min_tx_scn()), to_cstring(max_tx_scn_), - to_cstring(min_start_scn_), + to_cstring(get_min_start_scn()), inserted_cnt_, deleted_cnt_, write_ref_, - occupied_size_, + get_occupied_size(), last_insert_ts_, state_); fprintf(fd, "tx_data_count=%ld \n", tx_data_map_->count()); DumpTxDataMemtableFunctor fn(fd); - tx_data_map_->for_each(fn); + // tx_data_map_->for_each(fn); } if (NULL != fd) { fprintf(fd, "end of tx data memtable\n"); @@ -746,68 +985,6 @@ int ObTxDataMemtable::dump2text(const char *fname) return ret; } -void ObTxDataMemtable::DEBUG_dump_sort_list_node_2_text(const char *fname) -{ - int ret = OB_SUCCESS; - STORAGE_LOG(INFO, "start dump tx data sort list for debug"); - char real_fname[OB_MAX_FILE_NAME_LENGTH]; - FILE *fd = NULL; - - STORAGE_LOG(INFO, "dump2text for debug", K_(key)); - if (OB_ISNULL(fname)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "fanme is NULL"); - } else if (snprintf(real_fname, sizeof(real_fname), "%s.%ld", fname, - ::oceanbase::common::ObTimeUtility::current_time()) >= (int64_t)sizeof(real_fname)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "fname too long", K(fname)); - } else if (NULL == (fd = fopen(real_fname, "w"))) { - ret = OB_IO_ERROR; - STORAGE_LOG(WARN, "open file fail:", K(fname)); - } else { - auto ls_id = freezer_->get_ls_id().id(); - auto tenant_id = MTL_ID(); - fprintf(fd, "tenant_id=%ld ls_id=%ld\n", tenant_id, ls_id); - fprintf(fd, - "memtable: key=%s is_inited=%d is_iterating=%d has_constructed_list=%d min_tx_log_scn=%s max_tx_log_scn=%s " - "min_start_log_scn=%s inserted_cnt=%ld deleted_cnt=%ld write_ref=%ld occupied_size=%ld last_insert_ts=%ld " - "state=%d\n", - S(key_), - is_inited_, - is_iterating_, - has_constructed_list_, - to_cstring(min_tx_scn_), - to_cstring(max_tx_scn_), - to_cstring(min_start_scn_), - inserted_cnt_, - deleted_cnt_, - write_ref_, - occupied_size_, - last_insert_ts_, - state_); - fprintf(fd, "tx_data_count=%ld \n", tx_data_map_->count()); - DumpTxDataMemtableFunctor fn(fd); - - auto node = sort_list_head_.next_; - transaction::ObTransID unuse_key = 1; - while (OB_NOT_NULL(node)) { - auto tx_data = ObTxData::get_tx_data_by_sort_list_node(node); - node = node->next_; - - fn(unuse_key, tx_data); - } - } - - if (NULL != fd) { - fprintf(fd, "end of tx data memtable\n"); - fclose(fd); - fd = NULL; - } - if (OB_FAIL(ret)) { - STORAGE_LOG(WARN, "dump_tx_data_memtable fail", K(fname), K(ret)); - } -} - const char *ObTxDataMemtable::get_state_string() { STATIC_ASSERT(int(State::ACTIVE) == 0, "Invalid State Enum"); @@ -821,6 +998,153 @@ const char *ObTxDataMemtable::get_state_string() return STATE_TO_CHAR[int(state_)]; } +int ObTxDataMemtable::DEBUG_try_calc_upper_and_check_(ObCommitVersionsArray &merged_commit_versions) +{ + int ret = OB_SUCCESS; + + ObTxData *cur_node = get_sorted_list_head()->next_; + int64_t DEBUG_iter_cnt = 0; + while (OB_SUCC(ret) && OB_NOT_NULL(cur_node)) { + DEBUG_iter_cnt++; + ObTxData *tx_data = cur_node; + cur_node = cur_node->sort_list_node_.next_; + + if (ObTxData::COMMIT != tx_data->state_) { + continue; + } + + SCN upper_trans_version = SCN::min_scn(); + if (OB_FAIL(DEBUG_fake_calc_upper_trans_version(tx_data->start_scn_, upper_trans_version, merged_commit_versions))) { + STORAGE_LOG(ERROR, "invalid upper trans version", KR(ret)); + } else if (upper_trans_version < tx_data->commit_version_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "invalid upper trans version", KR(ret), K(upper_trans_version), KPC(tx_data)); + } + + if (OB_FAIL(ret)) { + DEBUG_print_start_scn_list_(); + DEBUG_print_merged_commit_versions_(merged_commit_versions); + } + } + if (OB_SUCC(ret) && DEBUG_iter_cnt != inserted_cnt_ - deleted_cnt_) { + ret = OB_SUCCESS; + STORAGE_LOG(ERROR, "invalid iter cnt", KR(ret), K(DEBUG_iter_cnt), K(inserted_cnt_), K(deleted_cnt_)); + } + return ret; +} + +int ObTxDataMemtable::DEBUG_fake_calc_upper_trans_version(const SCN sstable_end_scn, + SCN &upper_trans_version, + ObCommitVersionsArray &merged_commit_versions) +{ + int ret = OB_SUCCESS; + + ObIArray &array = merged_commit_versions.array_; + int l = 0; + int r = array.count() - 1; + + // Binary find the first start_log_ts that is greater than or equal to sstable_end_scn + while (l < r) { + int mid = (l + r) >> 1; + if (array.at(mid).start_scn_ < sstable_end_scn) { + l = mid + 1; + } else { + r = mid; + } + } + + // Check if the start_log_ts is greater than or equal to the sstable_end_scn. If not, delay the + // upper_trans_version calculation to the next time. + if (0 == array.count() || !array.at(l).commit_version_.is_valid()) { + upper_trans_version = SCN::max_scn(); + ret = OB_ERR_UNDEFINED; + STORAGE_LOG(WARN, "unexpected array count or commit version", K(array.count()), K(array.at(l))); + } else { + upper_trans_version = array.at(l).commit_version_; + } + + return ret; +} + +void ObTxDataMemtable::DEBUG_print_start_scn_list_() +{ + int ret = OB_SUCCESS; + const char *real_fname = "tx_data_start_log_ts_list"; + FILE *fd = NULL; + + if (NULL == (fd = fopen(real_fname, "w"))) { + ret = OB_IO_ERROR; + STORAGE_LOG(WARN, "open file fail:", K(real_fname)); + } else { + auto tenant_id = MTL_ID(); + fprintf(fd, "tenant_id=%ld \n", tenant_id); + ObTxData *cur_node = get_sorted_list_head()->next_; + while (OB_NOT_NULL(cur_node)) { + ObTxData *tx_data = cur_node; + cur_node = cur_node->sort_list_node_.next_; + + fprintf(fd, + "ObTxData : tx_id=%-19ld state=%-8s start_scn=%-19s " + "end_scn=%-19s " + "commit_version=%-19s\n", + tx_data->tx_id_.get_id(), + ObTxData::get_state_string(tx_data->state_), + to_cstring(tx_data->start_scn_), + to_cstring(tx_data->end_scn_), + to_cstring(tx_data->commit_version_)); + } + } + + if (NULL != fd) { + fprintf(fd, "end of start log ts list\n"); + fclose(fd); + fd = NULL; + } + if (OB_FAIL(ret)) { + STORAGE_LOG(WARN, "dump start scn list fail", K(real_fname), K(ret)); + } +} + +void ObTxDataMemtable::DEBUG_print_merged_commit_versions_(ObCommitVersionsArray &merged_commit_versions) +{ + int ret = OB_SUCCESS; + const auto &array = merged_commit_versions.array_; + const char *real_fname = "merge_commit_versions"; + FILE *fd = NULL; + + if (NULL == (fd = fopen(real_fname, "w"))) { + ret = OB_IO_ERROR; + STORAGE_LOG(WARN, "open file fail:", K(real_fname)); + } else { + auto tenant_id = MTL_ID(); + fprintf(fd, "tenant_id=%ld \n", tenant_id); + for (int i = 0; i < array.count(); i++) { + fprintf(fd, + "start_scn=%-19s " + "commit_version=%-19s\n", + to_cstring(array.at(i).start_scn_), + to_cstring(array.at(i).commit_version_)); + } + } + + if (NULL != fd) { + fprintf(fd, "end of commit versions array\n"); + fclose(fd); + fd = NULL; + } + if (OB_FAIL(ret)) { + STORAGE_LOG(WARN, "dump commit versions fail", K(real_fname), K(ret)); + } +} + +void ObTxDataMemtable::TEST_reset_tx_data_map_() +{ + int ret = OB_SUCCESS; + tx_data_map_ = nullptr; + init_tx_data_map_(); +} + + // ********************* Derived functions which are not supported ******************* int ObTxDataMemtable::get(const storage::ObTableIterParam ¶m, diff --git a/src/storage/tx_table/ob_tx_data_memtable.h b/src/storage/tx_table/ob_tx_data_memtable.h index 538e00fbc..8175a1e14 100644 --- a/src/storage/tx_table/ob_tx_data_memtable.h +++ b/src/storage/tx_table/ob_tx_data_memtable.h @@ -13,9 +13,12 @@ #ifndef OCEANBASE_STORAGE_OB_TX_DATA_MEMTABLE #define OCEANBASE_STORAGE_OB_TX_DATA_MEMTABLE +#include "share/scn.h" +#include "storage/checkpoint/ob_freeze_checkpoint.h" #include "storage/memtable/ob_memtable_interface.h" #include "storage/tx/ob_tx_data_define.h" -#include "share/scn.h" +#include "storage/tx_table/ob_tx_table_define.h" +#include "storage/tx_table/tx_table_local_buffer.h" namespace oceanbase { @@ -30,15 +33,82 @@ class ObTxDataTable; class ObTxDataMemtable : public memtable::ObIMemtable { private: + static int64_t PERIODICAL_SELECT_INTERVAL_NS; + + struct ProcessCommitVersionData + { + ProcessCommitVersionData(ObTxData *start_tx_data, + share::SCN cur_max_commit_version, + share::SCN pre_start_scn) + : cur_tx_data_(start_tx_data), + cur_max_commit_version_(cur_max_commit_version), + pre_start_scn_(pre_start_scn) {} + + ObTxData *cur_tx_data_; + share::SCN cur_max_commit_version_; + share::SCN pre_start_scn_; + // TODO : @gengli remove these variables + int64_t DEBUG_iter_commit_scn_cnt_; + share::SCN DEBUG_last_start_scn_; + }; + + struct TxDataFakeRowKey + { + public: + // default constructer for ObSEArray + TxDataFakeRowKey() {} + + TxDataFakeRowKey(const int64_t tx_id) + { + for (int64_t i = 0; i < OBJ_CNT; i++) { + obj_array_[i].set_int(tx_id); + } + rowkey_.assign(obj_array_, OBJ_CNT); + } + ~TxDataFakeRowKey() {} + + TO_STRING_KV(K_(rowkey), K_(obj_array)); + + void assign(const int64_t tx_id) + { + for (int64_t i = 0; i < OBJ_CNT; i++) { + obj_array_[i].set_int(tx_id); + } + rowkey_.assign(obj_array_, OBJ_CNT); + } + + const common::ObStoreRowkey &get_rowkey() const { return rowkey_; } + + const transaction::ObTransID get_tx_id() { return transaction::ObTransID(obj_array_[0].get_int()); } + + private: + static const int64_t OBJ_CNT = 1; + + public: + common::ObStoreRowkey rowkey_; + ObObj obj_array_[OBJ_CNT]; + }; + + struct TxId2CntPair { + TxId2CntPair() : tx_id_(0), tx_data_count_(0) {} + TxId2CntPair(const transaction::ObTransID tx_id, const int64_t tx_data_count) + : tx_id_(tx_id), tx_data_count_(tx_data_count) {} + transaction::ObTransID tx_id_; + int64_t tx_data_count_; + + TO_STRING_KV(K_(tx_id), K_(tx_data_count)); + }; + using SliceAllocator = ObSliceAlloc; - static const int MAX_TX_DATA_TABLE_CONCURRENCY = 64; + static const int MAX_TX_DATA_TABLE_CONCURRENCY = 1 << 6; // 64 + static const int MAX_CONCURRENCY_MOD_MASK = MAX_TX_DATA_TABLE_CONCURRENCY - 1; public: // active : freeze_ts is not set // freezing : freeze_ts is set, tx data is incomplete // frozen : tx data is complete and sorted // dumped : the memtable has been dumped - enum class State + enum class State : int32_t { ACTIVE = 0, FREEZING = 1, @@ -48,32 +118,28 @@ public: STATE_CNT }; - // temporary logic to do self-freeze - static const int64_t TX_DATA_MEMTABLE_SELF_FREEZE_THRESHOLD_CNT = 2000000; - public: // ObTxDataMemtable ObTxDataMemtable() : ObIMemtable(), is_inited_(false), is_iterating_(false), - has_constructed_list_(false), - min_tx_scn_(), + construct_list_done_(false), + pre_process_done_(false), max_tx_scn_(), - min_start_scn_(), inserted_cnt_(0), deleted_cnt_(0), write_ref_(0), - occupied_size_(0), + occupied_size_(), last_insert_ts_(0), state_(ObTxDataMemtable::State::INVALID), + arena_allocator_(), sort_list_head_(), tx_data_map_(nullptr), slice_allocator_(nullptr), memtable_mgr_(nullptr), - freezer_(nullptr) - { - reset_thread_local_list_(); - } + freezer_(nullptr), + buf_(arena_allocator_), + row_key_array_() {} ~ObTxDataMemtable() { reset(); } void reset(); int init(const ObITable::TableKey &table_key, @@ -95,12 +161,7 @@ public: // ObTxDataMemtable */ int get_tx_data(const transaction::ObTransID &tx_id, ObTxDataGuard &tx_data_guard); - /** - * @brief Get the tx data without guard - */ - int get_tx_data(const transaction::ObTransID &tx_id, ObTxData *&tx_data); - - void revert_tx_data(ObTxData *tx_data) { tx_data_map_->revert(tx_data); } + // void revert_tx_data(ObTxData *tx_data) { tx_data_map_->revert(tx_data); } /** * @brief This function is used by ObTxDataMemtableScanIterator and it will do the following @@ -110,32 +171,12 @@ public: // ObTxDataMemtable */ int prepare_tx_data_list(); - /** - * @brief This function is used by ObTxDataMemtableScanIterator after all tx data is dumped. It - * performs sorting similarly to prepare_tx_data_list() function by start_scn of tx data - * - */ - int prepare_commit_scn_list(); - /** * @brief Check if this tx data memtable can be minor merge * See more details at ready_for_flush() function. */ bool can_be_minor_merged() override; - /** - * @brief check if this tx data memtable contains a tx data - * - * @param tx_id the tx_id of tx data - */ - bool contain_tx_data(transaction::ObTransID tx_id); - - bool can_iterate() - { - bool bool_ret = (false == ATOMIC_CAS(&is_iterating_, false, true)); - return bool_ret; - } - /** * @brief delete a tx data from tx data memtable * @@ -143,7 +184,9 @@ public: // ObTxDataMemtable */ int remove(transaction::ObTransID tx_id); - bool need_self_freeze(); + int pre_process_for_merge(); + + int get_tx_data_cnt_by_tx_id(const transaction::ObTransID &tx_id, int64_t &tx_data_count); /** * @brief dump tx data memtable to file @@ -152,24 +195,26 @@ public: // ObTxDataMemtable */ int dump2text(const char *fname); - void DEBUG_dump_sort_list_node_2_text(const char *fname); - INHERIT_TO_STRING_KV("ObITable", ObITable, - KP(this), K_(is_inited), K_(is_iterating), - K_(has_constructed_list), - K_(min_tx_scn), + K_(pre_process_done), + K_(construct_list_done), + "min_tx_scn", get_min_tx_scn(), K_(max_tx_scn), - K_(min_start_scn), + "min_start_scn", get_min_start_scn(), K_(snapshot_version), K_(inserted_cnt), + K_(deleted_cnt), K_(write_ref), K_(occupied_size), K_(state), KP_(tx_data_map), - KP_(memtable_mgr)); + KP_(memtable_mgr), + K_(commit_versions_serialize_size), + K_(row_key_array), + K_(tx_id_2_cnt)); public: /* derived from ObITable */ @@ -209,8 +254,19 @@ public: /* derived from ObITable */ virtual bool is_frozen_memtable() const { return ObTxDataMemtable::State::FROZEN == state_; } public: /* derived from ObIMemtable */ - virtual int64_t get_occupied_size() const { return occupied_size_; } + virtual int64_t get_occupied_size() const + { + int64_t res = 0; + for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { + res += occupied_size_[i]; + } + return res; + } + virtual int get_split_ranges(const ObStoreRowkey *start_key, + const ObStoreRowkey *end_key, + const int64_t part_cnt, + common::ObIArray &range_array) override; // not supported virtual int get(const storage::ObTableIterParam ¶m, storage::ObTableAccessContext &context, @@ -239,9 +295,10 @@ public: /* derived from ObIMemtable */ const blocksstable::ObDatumRowkey &rowkey) override; public: // checkpoint - share::SCN get_rec_scn(); - - // int freeze(); + share::SCN get_rec_scn() + { + return get_min_tx_scn(); + } int flush(); @@ -253,7 +310,6 @@ public: // checkpoint bool ready_for_flush(); public: // getter && setter - share::SCN get_min_start_scn() { return min_start_scn_.atomic_get(); } int64_t get_tx_data_count() { return tx_data_map_->count(); } int64_t size() { return get_tx_data_count(); } int64_t get_inserted_count() { return inserted_cnt_; } @@ -262,17 +318,39 @@ public: // getter && setter int64_t dec_write_ref() { return ATOMIC_AAF(&write_ref_, -1); } int64_t get_write_ref() const override { return ATOMIC_LOAD(&write_ref_); } ObTxDataMemtable::State get_state() { return state_; } - ObTxDataSortListNode *get_sorted_list_head() { return &sort_list_head_; } + ObTxDataLinkNode *get_sorted_list_head() { return &sort_list_head_; } const char* get_state_string(); ObTxDataMemtableMgr *get_tx_data_memtable_mgr() { return memtable_mgr_; } - share::SCN get_min_tx_scn() { return min_tx_scn_; } + share::SCN get_min_tx_scn() const + { + share::SCN res = share::SCN::max_scn(); + for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { + share::SCN min_tx_scn = min_tx_scn_[i].atomic_load(); + if (min_tx_scn < res) { + res = min_tx_scn; + } + } + return res; + } + + share::SCN get_min_start_scn() const + { + share::SCN res = share::SCN::max_scn(); + for (int i = 0; i < MAX_TX_DATA_TABLE_CONCURRENCY; i++) { + share::SCN min_start_scn = min_start_scn_[i].atomic_load(); + if (min_start_scn < res) { + res = min_start_scn; + } + } + return res; + } share::SCN get_max_tx_scn() { return max_tx_scn_; } + int set_freezer(ObFreezer *handler); void set_start_scn(const share::SCN start_scn) {key_.scn_range_.start_scn_ = start_scn; } void set_end_scn() { key_.scn_range_.end_scn_ = max_tx_scn_; } void set_state(const ObTxDataMemtable::State &state) { state_ = state; } - void set_has_constructed_list(bool val) { has_constructed_list_ = val; } void reset_is_iterating() { ATOMIC_STORE(&is_iterating_, false); } @@ -280,41 +358,80 @@ public: // getter && setter private: // ObTxDataMemtable + void atomic_update_(ObTxData *tx_data); int do_sort_by_tx_id_(); int do_sort_by_start_scn_(); - void merge_sort_(int64_t (*get_key)(const ObTxData &), ObTxDataSortListNode *&head); - - ObTxDataSortListNode *quick_sort_(int64_t (*get_key)(const ObTxData &), ObTxDataSortListNode *head); - - ObTxDataSortListNode *merge_sorted_list_(int64_t (*get_key)(const ObTxData &), - ObTxDataSortListNode *left_list, - ObTxDataSortListNode *right_list); - void split_list_(ObTxDataSortListNode *head, - ObTxDataSortListNode *&left_list, - ObTxDataSortListNode *&right_list); - int cmp_key_(const int64_t &lhs, const int64_t &rhs); int construct_list_for_sort_(); + int init_tx_data_map_(); + + int pre_process_commit_version_row_(ObTxData *fake_tx_data); + + int insert_fake_tx_data_to_list_and_map_(ObTxData *fake_tx_data); + + int fill_in_cur_commit_versions_(ObCommitVersionsArray &cur_commit_versions); + + int periodical_get_next_commit_version_(ProcessCommitVersionData &process_data, + ObCommitVersionsArray::Node &node); + + int get_past_commit_versions_(ObCommitVersionsArray &past_commit_versions); + + int merge_cur_and_past_commit_verisons_(const share::SCN recycle_scn, + ObCommitVersionsArray &cur_commit_versions, + ObCommitVersionsArray &past_commit_versions, + ObCommitVersionsArray &merged_commit_versions); + + int push_range_bounds_(const int64_t part_cnt); + + int prepare_array_space_(const int64_t part_cnt); + void reset_thread_local_list_(); - int DEBUG_check_sort_result_(int64_t (*get_key)(const ObTxData &)); + void init_arena_allocator_(); + + void merge_sort_(int64_t (*get_key)(const ObTxData &), ObTxData *&head); + + ObTxData *merge_sorted_list_(int64_t (*get_key)(const ObTxData &), + ObTxData *left_list, + ObTxData *right_list); + void split_list_(ObTxData *head, + ObTxData *&left_list, + ObTxData *&right_list); + + ObTxDataLinkNode *quick_sort_(int64_t (*get_key)(const ObTxData &), + ObTxDataLinkNode *head); + + int merge_pre_process_node_(const int64_t step_len, + const share::SCN start_scn_limit, + const share::SCN recycle_scn, + const ObIArray &data_arr, + share::SCN &max_commit_version, + ObIArray &merged_arr); + + int DEBUG_try_calc_upper_and_check_(ObCommitVersionsArray &merged_commit_versions); + int DEBUG_fake_calc_upper_trans_version(const share::SCN sstable_end_scn, + share::SCN &upper_trans_version, + ObCommitVersionsArray &merged_commit_versions); + void DEBUG_print_start_scn_list_(); + void DEBUG_print_merged_commit_versions_(ObCommitVersionsArray &merged_commit_versions); + void TEST_reset_tx_data_map_(); + private: // ObTxDataMemtable bool is_inited_; bool is_iterating_; - bool has_constructed_list_; - - // the minimum scn of commit_version in this tx data memtable - share::SCN min_tx_scn_; + bool construct_list_done_; + bool pre_process_done_; // the maximum scn in this tx data memtable share::SCN max_tx_scn_; - - // the minimum start scn in this tx data memtable - share::SCN min_start_scn_; + // the minimum log ts of commit_log_ts in this tx data memtable + share::SCN min_tx_scn_[MAX_TX_DATA_TABLE_CONCURRENCY]; + // the minimum start log ts in this tx data memtable + share::SCN min_start_scn_[MAX_TX_DATA_TABLE_CONCURRENCY]; int64_t inserted_cnt_; @@ -322,7 +439,7 @@ private: // ObTxDataMemtable int64_t write_ref_; - int64_t occupied_size_; + int64_t occupied_size_[MAX_TX_DATA_TABLE_CONCURRENCY]; int64_t last_insert_ts_; @@ -330,11 +447,10 @@ private: // ObTxDataMemtable // active, freezing, frozen, dumped ObTxDataMemtable::State state_; + ObArenaAllocator arena_allocator_; + // the head node of sorted list which is used before dump - ObTxDataSortListNode sort_list_head_; - // use thread local list instead of foreach of link hash map can speed up constructing list for - // sort. - ObTxDataSortListNode local_sort_list_head_[MAX_TX_DATA_TABLE_CONCURRENCY]; + ObTxDataLinkNode sort_list_head_; // the hash map sotres tx data TxDataMap *tx_data_map_; @@ -348,25 +464,19 @@ private: // ObTxDataMemtable // used for checkpoint executor storage::ObFreezer *freezer_; -}; -class IterateTxDataMapForSortFunctor -{ -public: - explicit IterateTxDataMapForSortFunctor(ObTxDataSortListNode *head) : pre_sort_list_node_(head) {} + int64_t commit_versions_serialize_size_; + ObTxLocalBuffer buf_; - bool operator()(const transaction::ObTransID &key, ObTxData *tx_data) - { - UNUSED(key); - ObTxDataSortListNode *sort_list_node = ObTxData::get_sort_list_node_by_tx_data(tx_data); - pre_sort_list_node_->next_ = sort_list_node; - pre_sort_list_node_ = sort_list_node; + // provide ObStoreRowkey for get_split_range() function + ObSEArray row_key_array_; - return true; - } - -private: - ObTxDataSortListNode *pre_sort_list_node_; + // When parallel dump is enabled, the tx data sort list is splited to multiple ranges. This array map the start tx_id + // of the range to a tx data count in the range. Then the ObTxDataMemtableScanIterator can detect how many tx data + // need to be dumped. + ObSEArray tx_id_2_cnt_; + int64_t DEBUG_iter_commit_ts_cnt_; + share::SCN DEBUG_last_start_scn_; }; class DumpTxDataMemtableFunctor @@ -374,15 +484,13 @@ class DumpTxDataMemtableFunctor public: explicit DumpTxDataMemtableFunctor(FILE *fd) : fd_(fd) {} - bool operator()(const transaction::ObTransID &key, ObTxData *tx_data) { - UNUSED(key); + bool operator()(ObTxData *tx_data) { // printf basic info fprintf(fd_, - "ObTxData : tx_id=%-19ld is_in_memtable=%-3d state=%-8s start_scn=%-19s " + "ObTxData : tx_id=%-19ld state=%-8s start_scn=%-19s " "end_scn=%-19s " "commit_version=%-19s ", tx_data->tx_id_.get_id(), - tx_data->is_in_tx_data_table_, ObTxData::get_state_string(tx_data->state_), to_cstring(tx_data->start_scn_), to_cstring(tx_data->end_scn_), @@ -407,16 +515,6 @@ private: FILE *fd_; }; -OB_INLINE share::SCN ObTxDataMemtable::get_rec_scn() -{ - // TODO : @gengli - // rec_scn changes constantly. The rec_scn obtained by checkpoint mgr - // may be greater than the actual checkpoint of tx_data_memtable because the - // callback functions are not sequential. The checkpoint is determined both on - // the max-sequential callback point of the log and the rec_scn. - return min_tx_scn_; -} - } // namespace storage } // namespace oceanbase diff --git a/src/storage/tx_table/ob_tx_data_memtable_mgr.cpp b/src/storage/tx_table/ob_tx_data_memtable_mgr.cpp old mode 100644 new mode 100755 index 6b389aab7..4ec35b27d --- a/src/storage/tx_table/ob_tx_data_memtable_mgr.cpp +++ b/src/storage/tx_table/ob_tx_data_memtable_mgr.cpp @@ -461,7 +461,6 @@ bool ObTxDataMemtableMgr::is_flushing() const int ObTxDataMemtableMgr::get_memtable_range(int64_t &memtable_head, int64_t &memtable_tail) { int ret = OB_SUCCESS; - MemMgrRLockGuard lock_guard(lock_); memtable_head = memtable_head_; memtable_tail = memtable_tail_; return ret; diff --git a/src/storage/tx_table/ob_tx_data_table.cpp b/src/storage/tx_table/ob_tx_data_table.cpp index 67e78f6c7..6fb935d95 100644 --- a/src/storage/tx_table/ob_tx_data_table.cpp +++ b/src/storage/tx_table/ob_tx_data_table.cpp @@ -18,6 +18,7 @@ #include "storage/ls/ob_ls_tablet_service.h" #include "storage/tablet/ob_tablet_iterator.h" #include "storage/tx/ob_tx_data_functor.h" +#include "storage/tx/ob_tx_data_define.h" #include "storage/tx_storage/ob_tenant_freezer.h" #include "storage/tx_table/ob_tx_ctx_table.h" #include "storage/tx_table/ob_tx_table_define.h" @@ -36,30 +37,25 @@ namespace storage int ObTxDataTable::init(ObLS *ls, ObTxCtxTable *tx_ctx_table) { int ret = OB_SUCCESS; + STATIC_ASSERT(sizeof(ObTxData) <= TX_DATA_SLICE_SIZE, "Size of ObTxData Overflow."); + STATIC_ASSERT(sizeof(ObUndoAction) == UNDO_ACTION_SZIE, "Size of ObUndoAction Overflow."); + STATIC_ASSERT(sizeof(ObUndoStatusNode) <= TX_DATA_SLICE_SIZE, "Size of ObUndoStatusNode Overflow"); - STATIC_ASSERT(sizeof(ObTxData) <= TX_DATA_SIZE, "Size of ObTxData Overflow."); - STATIC_ASSERT(sizeof(TxDataHashNode) <= TX_DATA_HASH_NODE_SIZE, - "Size of TxDataHashNode Overflow."); - STATIC_ASSERT(sizeof(ObTxDataSortListNode) <= TX_DATA_SORT_LIST_NODE_SIZE, - "Size of ObTxDataSortListNode Overflow."); - STATIC_ASSERT(TX_DATA_SIZE + TX_DATA_HASH_NODE_SIZE + TX_DATA_SORT_LIST_NODE_SIZE - <= TX_DATA_SLICE_SIZE, - "Size of ObTxDataSortListNode Overflow."); - - mem_attr_.label_ = "TX_DATA_TABLE"; - mem_attr_.tenant_id_ = MTL_ID(); - mem_attr_.ctx_id_ = ObCtxIds::DEFAULT_CTX_ID; + ObMemAttr mem_attr; + mem_attr.label_ = "TX_DATA_TABLE"; + mem_attr.tenant_id_ = MTL_ID(); + mem_attr.ctx_id_ = ObCtxIds::DEFAULT_CTX_ID; ObMemtableMgrHandle memtable_mgr_handle; if (OB_ISNULL(ls) || OB_ISNULL(tx_ctx_table)) { ret = OB_ERR_NULL_VALUE; STORAGE_LOG(WARN, "ls tablet service or tx ctx table is nullptr", KR(ret)); } else if (OB_FAIL(slice_allocator_.init(TX_DATA_SLICE_SIZE, OB_MALLOC_NORMAL_BLOCK_SIZE, - common::default_blk_alloc, mem_attr_))) { + common::default_blk_alloc, mem_attr))) { STORAGE_LOG(ERROR, "slice_allocator_ init fail"); } else if (FALSE_IT(ls_tablet_svr_ = ls->get_tablet_svr())) { } else if (OB_FAIL(ls_tablet_svr_->get_tx_data_memtable_mgr(memtable_mgr_handle))) { STORAGE_LOG(WARN, "get tx data memtable mgr fail.", KR(ret), K(tablet_id_)); - } else if (FALSE_IT(arena_allocator_.set_attr(mem_attr_))) { + } else if (FALSE_IT(arena_allocator_.set_attr(mem_attr))) { } else if (OB_FAIL(init_tx_data_read_schema_())) { STORAGE_LOG(WARN, "init tx data read ctx failed.", KR(ret), K(tablet_id_)); } else { @@ -72,6 +68,7 @@ int ObTxDataTable::init(ObLS *ls, ObTxCtxTable *tx_ctx_table) tablet_id_ = LS_TX_DATA_TABLET; is_inited_ = true; + FLOG_INFO("tx data table init success", K(sizeof(ObTxData)), K(sizeof(ObTxDataLinkNode)), KPC(this)); } return ret; } @@ -225,7 +222,7 @@ int ObTxDataTable::clean_memtables_cache_() void ObTxDataTable::destroy() { reset(); } -int ObTxDataTable::alloc_tx_data(ObTxData *&tx_data) +int ObTxDataTable::alloc_tx_data(ObTxDataGuard &tx_data_guard) { int ret = OB_SUCCESS; void *slice_ptr = nullptr; @@ -234,25 +231,34 @@ int ObTxDataTable::alloc_tx_data(ObTxData *&tx_data) STORAGE_LOG(WARN, "allocate memory from slice_allocator fail.", KR(ret), KP(this), K(tablet_id_)); } else { - tx_data = ObTxData::get_tx_data_by_hash_node(reinterpret_cast(slice_ptr)); - // construct ObTxData() - new (tx_data) ObTxData(); + ObTxData *tx_data = new (slice_ptr) ObTxData(); + tx_data->slice_allocator_ = &slice_allocator_; + tx_data_guard.init(tx_data); } return ret; } -int ObTxDataTable::deep_copy_tx_data(ObTxData *in_tx_data, ObTxData *&out_tx_data) +int ObTxDataTable::deep_copy_tx_data(const ObTxDataGuard &in_tx_data_guard, ObTxDataGuard &out_tx_data_guard) { int ret = OB_SUCCESS; void *slice_ptr = nullptr; + const ObTxData *in_tx_data = in_tx_data_guard.tx_data(); + ObTxData *out_tx_data = nullptr; if (OB_ISNULL(slice_ptr = slice_allocator_.alloc())) { ret = OB_ALLOCATE_MEMORY_FAILED; STORAGE_LOG(WARN, "allocate memory from slice_allocator fail.", KR(ret), KP(this), K(tablet_id_)); + } else if (OB_ISNULL(in_tx_data)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "invalid nullptr of tx data", K(in_tx_data_guard), KPC(this)); } else { - out_tx_data = ObTxData::get_tx_data_by_hash_node(reinterpret_cast(slice_ptr)); - new (out_tx_data) ObTxData(*in_tx_data); + out_tx_data = new (slice_ptr) ObTxData(); + *out_tx_data = *in_tx_data; + out_tx_data->slice_allocator_ = &slice_allocator_; out_tx_data->undo_status_list_.head_ = nullptr; + out_tx_data->ref_cnt_ = 0; + out_tx_data_guard.init(out_tx_data); + if (OB_FAIL(deep_copy_undo_status_list_(in_tx_data->undo_status_list_, out_tx_data->undo_status_list_))) { STORAGE_LOG(WARN, "deep copy undo status list failed."); @@ -263,7 +269,7 @@ int ObTxDataTable::deep_copy_tx_data(ObTxData *in_tx_data, ObTxData *&out_tx_dat return ret; } -int ObTxDataTable::deep_copy_undo_status_list_(ObUndoStatusList &in_list, +int ObTxDataTable::deep_copy_undo_status_list_(const ObUndoStatusList &in_list, ObUndoStatusList &out_list) { int ret = OB_SUCCESS; @@ -302,16 +308,6 @@ int ObTxDataTable::alloc_undo_status_node(ObUndoStatusNode *&undo_status_node) return ret; } -void ObTxDataTable::free_tx_data(ObTxData *tx_data) -{ - if (OB_NOT_NULL(tx_data)) { - free_undo_status_list_(tx_data->undo_status_list_.head_); - // The memory of tx data belongs to a slice of memory allocated by slice allocator. And the - // start of the slice memory is hash node. - slice_allocator_.free(ObTxData::get_hash_node_by_tx_data(tx_data)); - } -} - int ObTxDataTable::free_undo_status_node(ObUndoStatusNode *&undo_status_node) { int ret = OB_SUCCESS; @@ -319,7 +315,7 @@ int ObTxDataTable::free_undo_status_node(ObUndoStatusNode *&undo_status_node) ret = OB_ERR_NULL_VALUE; STORAGE_LOG(WARN, "trying to free nullptr", KR(ret), K(tablet_id_)); } else { - slice_allocator_.free(reinterpret_cast(undo_status_node)); + slice_allocator_.free(undo_status_node); } return ret; } @@ -360,8 +356,8 @@ int ObTxDataTable::insert(ObTxData *&tx_data) } else if (OB_FAIL(insert_(tx_data, write_guard))) { STORAGE_LOG(WARN, "insert tx data failed.", KR(ret), KPC(tx_data), KP(this), K(tablet_id_)); } else { - // STORAGE_LOG(DEBUG, "insert tx data succeed.", KPC(tx_data)); // successfully insert + // TODO : @gengli do not dec ref and set nullptr after insert } if (tg.get_diff() > 100000) { STORAGE_LOG(INFO, "ObTxDataTable insert cost too much time", K(tx_id), K(tg)); @@ -377,6 +373,7 @@ int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write { int ret = OB_SUCCESS; common::ObTimeGuard tg("tx_data_table::insert_", 100 * 1000); + bool inserted = false; ObTxDataMemtable *tx_data_memtable = nullptr; ObTableHandleV2 (&memtable_handles)[MAX_TX_DATA_MEMTABLE_CNT] = write_guard.handles_; @@ -390,39 +387,42 @@ int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "tx data memtable is nullptr.", KR(ret), KP(this), K(tablet_id_), K(memtable_handles[i])); - } else if (OB_UNLIKELY(ObTxDataMemtable::State::RELEASED == tx_data_memtable->get_state())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, - "trying to insert a tx data into a tx data memtable in frozen/dumped state.", - KR(ret), KP(this), K(tablet_id_), KP(tx_data_memtable), KPC(tx_data_memtable)); } else if (FALSE_IT(tg.click())) { // do nothing } else if (tx_data_memtable->get_start_scn() < tx_data->end_scn_ && tx_data_memtable->get_end_scn() >= tx_data->end_scn_) { tg.click(); - ret = insert_into_memtable_(tx_data_memtable, tx_data); + if (OB_FAIL(tx_data_memtable->insert(tx_data))) { + STORAGE_LOG(WARN, + "insert tx data into tx data memtable failed", + KR(ret), + "ls_id", get_ls_id(), + KPC(tx_data), + KPC(tx_data_memtable)); + } else { + inserted = true; + } } else { // should not insert into this memtable + STORAGE_LOG(INFO, "skip this tx data memtable", KPC(tx_data), KPC(tx_data_memtable)); } } tg.click(); // If this tx data can not be inserted into all memtables, check if it should be filtered. - // We use the start scn of the first memtable as the filtering time stamp - if (OB_SUCC(ret) && OB_NOT_NULL(tx_data) && OB_NOT_NULL(tx_data_memtable)) { + // We use the start log ts of the first memtable as the filtering time stamp + if (OB_SUCC(ret) && !inserted && OB_NOT_NULL(tx_data_memtable)) { SCN clog_checkpoint_scn = tx_data_memtable->get_key().get_start_scn(); if (tx_data->end_scn_ <= clog_checkpoint_scn) { // Filter this tx data. The part trans ctx need to handle this error code because the memory // of tx data need to be freed. STORAGE_LOG(INFO, "This tx data is filtered.", K(clog_checkpoint_scn), KPC(tx_data)); - free_tx_data(tx_data); - tx_data = nullptr; tg.click(); } else { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "cannot find the correct tx data memtable to insert", KR(ret), - KPC(tx_data), K(memtable_handles)); + KPC(tx_data), K(clog_checkpoint_scn), K(memtable_handles)); } } if (tg.get_diff() > 100000) { @@ -432,69 +432,6 @@ int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write return ret; } -// A tx data with lesser end_scn may be inserted after a tx data with greater end_scn due to -// the out-of-order callback of log. This function will retain the newer tx data and delete the -// older one. -int ObTxDataTable::insert_into_memtable_(ObTxDataMemtable *tx_data_memtable, ObTxData *&tx_data) -{ - common::ObTimeGuard tg("tx_data_table::insert_into_memtable", 100 * 1000); - int ret = OB_SUCCESS; - bool need_insert = true; - - if (OB_UNLIKELY(tx_data_memtable->contain_tx_data(tx_data->tx_id_))) { - tg.click(); - // check and insert - ObTxData *existed_tx_data = nullptr; - if (OB_FAIL(tx_data_memtable->get_tx_data(tx_data->tx_id_, existed_tx_data))) { - STORAGE_LOG(WARN, "get tx data from tx data memtable failed.", KR(ret), KPC(tx_data), - KPC(tx_data_memtable)); - } else if (OB_ISNULL(existed_tx_data)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "existed tx data is unexpected nullptr", KR(ret), KPC(tx_data), - KPC(tx_data_memtable)); - } else if (existed_tx_data->end_scn_ < tx_data->end_scn_) { - tg.click(); - if (OB_FAIL(tx_data_memtable->remove(tx_data->tx_id_))) { - STORAGE_LOG(ERROR, "remove tx data from tx data memtable failed.", KR(ret), KPC(tx_data), - KPC(tx_data_memtable)); - } else { - existed_tx_data->is_in_tx_data_table_ = false; - - // successfully remove - } - } else { - // The end_scn of tx data in memtable is greater than or equal to the end_scn of current - // tx data, which means the tx data waiting to insert is an older one. So we set need_insert = - // false to skip inserting this tx data - need_insert = false; - tx_data_memtable->revert_tx_data(existed_tx_data); - } - } - tg.click(); - - // insert or free according to the need_insert flag - if (OB_SUCC(ret)) { - if (OB_LIKELY(need_insert)) { - tg.click(); - if (OB_FAIL(tx_data_memtable->insert(tx_data))) { - STORAGE_LOG(WARN, "insert tx data into tx data memtable failed.", KR(ret), - KPC(tx_data_memtable), KPC(tx_data)); - } else { - tx_data = nullptr; - } - tg.click(); - } else { - free_tx_data(tx_data); - tx_data = nullptr; - } - } - if (tg.get_diff() > 100000) { - STORAGE_LOG(INFO, "ObTxDataTable insert_info_memtable cost too much time", K(tg)); - } - - return ret; -} - int ObTxDataTable::check_with_tx_data(const ObTransID tx_id, ObITxDataCheckFunctor &fn) { int ret = OB_SUCCESS; @@ -519,52 +456,100 @@ int ObTxDataTable::check_tx_data_in_memtable_(const ObTransID tx_id, ObITxDataCh { int ret = OB_SUCCESS; - if (OB_FAIL(update_memtables_cache())) { - STORAGE_LOG(WARN, "get all memtables fail.", KR(ret), K(tx_id)); - } else { - // look for the tx data - // construct memtable_handle before tx_data_guard to destruct it latter - ObTableHandleV2 src_memtable_handle; - ObTxDataGuard tx_data_guard; - bool find = false; - - if (OB_FAIL(get_tx_data_in_memtables_cache_(tx_id, src_memtable_handle, tx_data_guard, find))) { - STORAGE_LOG(INFO, "get tx data in memtables cache failed.", KR(ret), K(tx_id)); - } else if (find) { - // do function if find tx data in memtable - if (OB_FAIL(fn(tx_data_guard.tx_data()))) { - STORAGE_LOG(WARN, "do data check function fail.", KR(ret), KP(this), K(tablet_id_), K(tx_data_guard.tx_data())); + while (OB_SUCC(ret)) { + if (OB_FAIL(check_tx_data_with_cache_once_(tx_id, fn))) { + if (OB_EAGAIN == ret) { + ret = OB_SUCCESS; + if (OB_FAIL(update_memtables_cache())) { + STORAGE_LOG(WARN, "update memtables cache failed", KR(ret)); + } else { + // do check_tx_data_with_cache_once_ again + } + } else if (OB_TRANS_CTX_NOT_EXIST == ret) { + // need check tx data in sstable } else { - // do data check function success + STORAGE_LOG(WARN, "check tx data with cache failed", KR(ret)); } } else { - ret = OB_TRANS_CTX_NOT_EXIST; + // check tx data with cache succeed + break; } } return ret; } +int ObTxDataTable::check_tx_data_with_cache_once_(const transaction::ObTransID tx_id, ObITxDataCheckFunctor &fn) +{ + int ret = OB_SUCCESS; + + ObTxDataMemtable *tx_data_memtable = nullptr; + ObTxDataGuard tx_data_guard; + bool find = false; + + TCRLockGuard guard(memtables_cache_.lock_); + ObTableHdlArray &memtable_handles = memtables_cache_.memtable_handles_; + + for (int i = memtable_handles.count() - 1; OB_SUCC(ret) && !find && i >= 0; i--) { + tx_data_memtable = nullptr; + if (OB_FAIL(memtable_handles.at(i).get_tx_data_memtable(tx_data_memtable))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "get tx data memtable from table handles fail.", KR(ret), K(tx_id), K(memtable_handles.at(i))); + } else if (OB_ISNULL(tx_data_memtable)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "tx data memtable is nullptr.", KR(ret), K(tx_id)); + } else { + int tmp_ret = tx_data_memtable->get_tx_data(tx_id, tx_data_guard); + if (OB_SUCCESS == tmp_ret) { + find = true; + } else if (OB_ENTRY_NOT_EXIST == tmp_ret) { + // search tx_id in next memtable + } else { + STORAGE_LOG(WARN, + "get_tx_data fail.", + KR(tmp_ret), + KP(this), + K(tablet_id_), + KP(tx_data_memtable), + KPC(tx_data_memtable)); + } + } + } + + if (OB_SUCC(ret)) { + if (find) { + ret = fn(*tx_data_guard.tx_data()); + } else { + int64_t memtable_head = -1; + int64_t memtable_tail = -1; + if (OB_FAIL(get_memtable_mgr_()->get_memtable_range(memtable_head, memtable_tail))) { + STORAGE_LOG(WARN, "get memtable range failed", KR(ret)); + } else if (memtable_head != memtables_cache_.memtable_head_ || memtable_tail != memtables_cache_.memtable_tail_) { + ret = OB_EAGAIN; + } else { + ret = OB_TRANS_CTX_NOT_EXIST; + } + } + } + return ret; +} + int ObTxDataTable::update_memtables_cache() { int ret = OB_SUCCESS; bool need_update = true; + + // lock for updating memtables cache + TCWLockGuard guard(memtables_cache_.lock_); if (OB_FAIL(check_need_update_memtables_cache_(need_update))) { STORAGE_LOG(WARN, "check if memtable handles need update failed.", KR(ret)); - } else if (need_update) { - bool make_sure_need_update = true; - // lock for updating memtables cache - TCWLockGuard guard(memtables_cache_.lock_); - if (OB_FAIL(check_need_update_memtables_cache_(make_sure_need_update))) { - STORAGE_LOG(WARN, "check if memtable handles need update failed.", KR(ret)); - } else if (!make_sure_need_update) { - // do not need update cache, skip update - } else if (FALSE_IT(memtables_cache_.reuse())) { - } else if (OB_FAIL(get_memtable_mgr_()->get_all_memtables_with_range(memtables_cache_.memtable_handles_, - memtables_cache_.memtable_head_, - memtables_cache_.memtable_tail_))) { - STORAGE_LOG(WARN, "get all memtables with range failed.", KR(ret), KPC(this), KPC(get_memtable_mgr_())); - } + } else if (!need_update) { + // do not need update cache, skip update + } else if (FALSE_IT(memtables_cache_.reuse())) { + } else if (OB_FAIL(get_memtable_mgr_()->get_all_memtables_with_range(memtables_cache_.memtable_handles_, + memtables_cache_.memtable_head_, + memtables_cache_.memtable_tail_))) { + STORAGE_LOG(WARN, "get all memtables with range failed.", KR(ret), KPC(this), KPC(get_memtable_mgr_())); } return ret; @@ -575,11 +560,7 @@ int ObTxDataTable::check_need_update_memtables_cache_(bool &need_update) int ret = OB_SUCCESS; int64_t memtable_head = -1; int64_t memtable_tail = -1; - if (!is_started_) { - ret = OB_NOT_RUNNING; - need_update = false; - STORAGE_LOG(WARN, "tx data memtable has stopped", KR(ret), KPC(this), K(need_update)); - } else if (OB_FAIL(get_memtable_mgr_()->get_memtable_range(memtable_head, memtable_tail))) { + if (OB_FAIL(get_memtable_mgr_()->get_memtable_range(memtable_head, memtable_tail))) { STORAGE_LOG(WARN, "get memtable range failed.", KR(ret)); } else if (memtables_cache_.memtable_head_ == memtable_head && memtables_cache_.memtable_tail_ == memtable_tail) { // cache already up to date, skip update @@ -615,6 +596,8 @@ int ObTxDataTable::get_tx_data_in_memtables_cache_(const ObTransID tx_id, } else if (OB_ISNULL(tx_data_memtable)) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "tx data memtable is nullptr.", KR(ret), K(tx_id)); + } else if (ObTxDataMemtable::State::RELEASED == tx_data_memtable->get_state()) { + // skip get tx data in this tx data memtable } else { int tmp_ret = tx_data_memtable->get_tx_data(tx_id, tx_data_guard); if (OB_SUCCESS == tmp_ret) { @@ -655,9 +638,9 @@ int ObTxDataTable::check_tx_data_in_sstable_(const ObTransID tx_id, ObITxDataChe STORAGE_LOG(WARN, "check tx data in sstable failed.", KR(ret), KP(this), K(tablet_id_)); } - // free undo status list if exist if (OB_NOT_NULL(tx_data.undo_status_list_.head_)) { free_undo_status_list_(tx_data.undo_status_list_.head_); + tx_data.undo_status_list_.head_ = nullptr; } return ret; } @@ -690,7 +673,6 @@ int ObTxDataTable::get_tx_data_in_sstable_(const transaction::ObTransID tx_id, O // get tx data from sstable succeed. } } - return ret; } @@ -831,11 +813,11 @@ int ObTxDataTable::get_upper_trans_version_before_given_scn(const SCN sstable_en if (OB_FAIL(ret)) { } else if (skip_calc) { - } else if (0 == calc_upper_trans_version_cache_.commit_scns_.array_.count()) { + } else if (0 == calc_upper_trans_version_cache_.commit_versions_.array_.count()) { STORAGE_LOG(ERROR, "Unexpected empty array.", K(calc_upper_trans_version_cache_)); } else { TCRLockGuard lock_guard(calc_upper_trans_version_cache_.lock_); - if (!calc_upper_trans_version_cache_.commit_scns_.is_valid()) { + if (!calc_upper_trans_version_cache_.commit_versions_.is_valid()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "invalid cache for upper trans version calculation", KR(ret)); } else if (OB_FAIL(calc_upper_trans_scn_(sstable_end_scn, upper_trans_version))) { @@ -1156,7 +1138,7 @@ int ObTxDataTable::update_calc_upper_trans_version_cache_(ObITable *table) LOG_WARN("invalid tablet handle", KR(ret), K(tablet_handle), K(tablet_id_)); } else { ObCommitVersionsGetter getter(iter_param, table); - if (OB_FAIL(getter.get_next_row(calc_upper_trans_version_cache_.commit_scns_))) { + if (OB_FAIL(getter.get_next_row(calc_upper_trans_version_cache_.commit_versions_))) { STORAGE_LOG(WARN, "update calc_upper_trans_trans_version_cache failed.", KR(ret), KPC(table)); } else { calc_upper_trans_version_cache_.is_inited_ = true; @@ -1171,7 +1153,7 @@ int ObTxDataTable::calc_upper_trans_scn_(const SCN sstable_end_scn, SCN &upper_t { int ret = OB_SUCCESS; - const auto &array = calc_upper_trans_version_cache_.commit_scns_.array_; + const auto &array = calc_upper_trans_version_cache_.commit_versions_.array_; int l = 0; int r = array.count() - 1; @@ -1203,10 +1185,13 @@ int ObTxDataTable::calc_upper_trans_scn_(const SCN sstable_end_scn, SCN &upper_t "ls_id", get_ls_id(), "array_count", array.count(), "chose_idx", l); + for (int i = 0; i < array.count(); i++) { + STORAGE_LOG(INFO, "commit versions array : ", K(i), K(array.at(i))); + } return ret; } -int ObTxDataTable::supplement_undo_actions_if_exist(ObTxData *&tx_data) +int ObTxDataTable::supplement_undo_actions_if_exist(ObTxData *tx_data) { int ret = OB_SUCCESS; ObTxData tx_data_from_sstable; @@ -1228,6 +1213,12 @@ int ObTxDataTable::supplement_undo_actions_if_exist(ObTxData *&tx_data) } else { // assign and reset to avoid deep copy tx_data->undo_status_list_ = tx_data_from_sstable.undo_status_list_; + tx_data_from_sstable.undo_status_list_.reset(); + } + + if (OB_NOT_NULL(tx_data_from_sstable.undo_status_list_.head_)) { + STORAGE_LOG(WARN, "supplement undo actions failed", KR(ret), KPC(tx_data), K(get_ls_id())); + free_undo_status_list_(tx_data_from_sstable.undo_status_list_.head_); } return ret; } @@ -1292,13 +1283,14 @@ int ObTxDataTable::dump_tx_data_in_memtable_2_text_(const ObTransID tx_id, FILE } else { ObTableHandleV2 src_memtable_handle; ObTxDataGuard tx_data_guard; + tx_data_guard.reset(); bool find = false; if (OB_FAIL(get_tx_data_in_memtables_cache_(tx_id, src_memtable_handle, tx_data_guard, find))) { STORAGE_LOG(INFO, "get tx data in memtables cache failed.", KR(ret), K(tx_id)); } else if (find) { fprintf(fd, "********** Tx Data MemTable ***********\n\n"); - tx_data_guard.tx_data().dump_2_text(fd); + tx_data_guard.tx_data()->dump_2_text(fd); fprintf(fd, "\n********** Tx Data MemTable ***********\n"); } else { ret = OB_TRANS_CTX_NOT_EXIST; @@ -1325,17 +1317,6 @@ int ObTxDataTable::dump_tx_data_in_sstable_2_text_(const ObTransID tx_id, FILE * share::ObLSID ObTxDataTable::get_ls_id() { return ls_->get_ls_id(); } -bool CleanTxDataSSTableCacheFunctor::operator()(const transaction::ObTransID &key, - ObTxData *tx_data) -{ - int ret = OB_SUCCESS; - int64_t *latest_used_ts = ObTxData::get_latest_used_ts_by_tx_data(tx_data); - if ((*latest_used_ts < clean_ts_) && (OB_FAIL(sstable_cache_.del(key)))) { - STORAGE_LOG(WARN, "delete tx data from tx data sstable cache failed.", KR(ret), KPC(tx_data)); - } - return true; -} - } // namespace storage } // namespace oceanbase diff --git a/src/storage/tx_table/ob_tx_data_table.h b/src/storage/tx_table/ob_tx_data_table.h index 5214045d7..521830bdc 100644 --- a/src/storage/tx_table/ob_tx_data_table.h +++ b/src/storage/tx_table/ob_tx_data_table.h @@ -15,6 +15,7 @@ #include "storage/meta_mem/ob_tablet_handle.h" #include "lib/future/ob_future.h" +#include "share/scn.h" #include "storage/tx_table/ob_tx_data_memtable_mgr.h" #include "storage/tx_table/ob_tx_table_define.h" #include "share/ob_occam_timer.h" @@ -109,8 +110,8 @@ public: // The tx data memtable cannot freeze it self if its memory use is less than 1% static constexpr double TX_DATA_FREEZE_TRIGGER_MIN_PERCENTAGE = 1; - // The tx data memtable will trigger a freeze if its memory use is more than 10% - static constexpr double TX_DATA_FREEZE_TRIGGER_MAX_PERCENTAGE = 10; + // The tx data memtable will trigger a freeze if its memory use is more than 5% + static constexpr double TX_DATA_FREEZE_TRIGGER_MAX_PERCENTAGE = 5; enum COLUMN_ID_LIST { @@ -126,7 +127,6 @@ public: // ObTxDataTable : is_inited_(false), is_started_(false), tablet_id_(0), - mem_attr_(), slice_allocator_(), arena_allocator_(), ls_(nullptr), @@ -151,7 +151,7 @@ public: // ObTxDataTable * * @param[out] tx_data the tx data allocated by slice allocator */ - virtual int alloc_tx_data(ObTxData *&tx_data); + virtual int alloc_tx_data(ObTxDataGuard &tx_data); /** * @brief allocate memory and deep copy tx data @@ -159,14 +159,7 @@ public: // ObTxDataTable * @param[in] in_tx_data input tx data * @param[out] out_tx_data output tx data */ - virtual int deep_copy_tx_data(ObTxData *in_tx_data, ObTxData *&out_tx_data); - - /** - * @brief Free tx data with slice allocator - * - * @param[out] tx_data the tx data need to be freed - */ - virtual void free_tx_data(ObTxData *tx_data); + virtual int deep_copy_tx_data(const ObTxDataGuard &in_tx_data, ObTxDataGuard &out_tx_data); /** * @brief In order to reduce memory fragmentation and improve memory reuse rate,the variable @@ -215,7 +208,7 @@ public: // ObTxDataTable /** * @brief see ObTxTable::supplement_undo_actions_if_exist */ - int supplement_undo_actions_if_exist(ObTxData *&tx_data); + int supplement_undo_actions_if_exist(ObTxData *tx_data); int self_freeze_task(); @@ -260,6 +253,7 @@ private: int register_clean_cache_task_(); int check_tx_data_in_memtable_(const transaction::ObTransID tx_id, ObITxDataCheckFunctor &fn); + int check_tx_data_with_cache_once_(const transaction::ObTransID tx_id, ObITxDataCheckFunctor &fn); int check_tx_data_in_sstable_(const transaction::ObTransID tx_id, ObITxDataCheckFunctor &fn); @@ -274,7 +268,7 @@ private: // free the whole undo status list allocated by slice allocator int get_min_end_scn_from_single_tablet_(ObTabletHandle &tablet_handle, share::SCN &end_scn); - int deep_copy_undo_status_list_(ObUndoStatusList &in_list, ObUndoStatusList &out_list); + int deep_copy_undo_status_list_(const ObUndoStatusList &in_list, ObUndoStatusList &out_list); int init_tx_data_read_schema_(); int update_cache_if_needed_(bool &skip_calc); @@ -311,31 +305,11 @@ private: int check_min_start_in_tx_data_(const share::SCN &sstable_end_scn, share::SCN &min_start_ts_in_tx_data_memtable, bool &need_skip); - void print_alloc_size_for_test_(); - // free the whole undo status list allocated by slice allocator void free_undo_status_list_(ObUndoStatusNode *node_ptr); - void clean_sstable_cache_task_(int64_t cache_keeped_time); void update_calc_upper_info_(const share::SCN &max_decided_log_ts); - - void TEST_print_alloc_size_() - { - int tx_data_size = sizeof(ObTxData); - int tx_data_hash_node_size = sizeof(TxDataHashNode); - int tx_data_sort_list_node_size = sizeof(ObTxDataSortListNode); - int tx_data_undostatus_node_size = sizeof(ObUndoStatusNode); - int undo_status_list_node_size = sizeof(ObUndoStatusList); - int list_node_head_size = sizeof(ObUndoStatusList::head_); - int list_node_lock_size = sizeof(ObUndoStatusList::lock_); - int total_size_in_slice = tx_data_size + tx_data_hash_node_size + tx_data_sort_list_node_size; - STORAGETEST_LOG(INFO, "", K(tx_data_size), K(tx_data_hash_node_size), - K(tx_data_sort_list_node_size), K(total_size_in_slice), - K(tx_data_undostatus_node_size), K(undo_status_list_node_size), - K(list_node_head_size), K(list_node_lock_size)); - } - private: static const int64_t LS_TX_DATA_SCHEMA_VERSION = 0; static const int64_t LS_TX_DATA_SCHEMA_ROWKEY_CNT = 2; @@ -343,7 +317,6 @@ private: bool is_inited_; bool is_started_; ObTabletID tablet_id_; - ObMemAttr mem_attr_; // Allocator to allocate ObTxData and ObUndoStatus SliceAllocator slice_allocator_; ObArenaAllocator arena_allocator_; diff --git a/src/storage/tx_table/ob_tx_table.cpp b/src/storage/tx_table/ob_tx_table.cpp index 5e9e33143..a3c84c465 100644 --- a/src/storage/tx_table/ob_tx_table.cpp +++ b/src/storage/tx_table/ob_tx_table.cpp @@ -794,24 +794,24 @@ void ObTxTable::destroy() is_inited_ = false; } -int ObTxTable::alloc_tx_data(ObTxData *&tx_data) { +int ObTxTable::alloc_tx_data(ObTxDataGuard &tx_data_guard) { int ret = OB_SUCCESS; if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("tx table is not init.", KR(ret)); - } else if (OB_FAIL(tx_data_table_.alloc_tx_data(tx_data))) { + } else if (OB_FAIL(tx_data_table_.alloc_tx_data(tx_data_guard))) { LOG_WARN("allocate tx data from tx data table fail.", KR(ret)); } return ret; } -int ObTxTable::deep_copy_tx_data(ObTxData *in_tx_data, ObTxData *&out_tx_data) +int ObTxTable::deep_copy_tx_data(const ObTxDataGuard &in_tx_data_guard, ObTxDataGuard &out_tx_data_guard) { int ret = OB_SUCCESS; if (IS_NOT_INIT) { ret = OB_NOT_INIT; LOG_WARN("tx table is not init.", KR(ret)); - } else if (OB_FAIL(tx_data_table_.deep_copy_tx_data(in_tx_data, out_tx_data))) { + } else if (OB_FAIL(tx_data_table_.deep_copy_tx_data(in_tx_data_guard, out_tx_data_guard))) { LOG_WARN("deep copy tx data from tx data table fail", KR(ret)); } return ret; @@ -979,19 +979,30 @@ int ObTxTable::lock_for_read(const transaction::ObLockForReadArg &lock_for_read_ return ret; } -int ObTxTable::get_recycle_scn(SCN &recycle_scn) +int ObTxTable::get_recycle_scn(SCN &real_recycle_scn) { int ret = OB_SUCCESS; + real_recycle_scn = SCN::min_scn(); int64_t prev_epoch = ATOMIC_LOAD(&epoch_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "this tx table is not inited", KR(ret)); - } else if (OB_FAIL(tx_data_table_.get_recycle_scn(recycle_scn))) { - STORAGE_LOG(WARN, "get recycle ts failed", KR(ret), "ls_id", ls_->get_ls_id()); + SCN tablet_recycle_scn = SCN::min_scn(); + SCN delay_recycle_scn = SCN::max_scn(); + if (OB_FAIL(tx_data_table_.get_recycle_scn(tablet_recycle_scn))) { + TRANS_LOG(WARN, "get recycle scn from tx data table failed.", KR(ret)); } else if (TxTableState::ONLINE != ATOMIC_LOAD(&state_) || prev_epoch != ATOMIC_LOAD(&epoch_)) { - recycle_scn.set_min(); + real_recycle_scn = SCN::min_scn(); ret = OB_REPLICA_NOT_READABLE; STORAGE_LOG(WARN, "this tx table is migrating or has migrated", KR(ret), "ls_id", ls_->get_ls_id()); + } else { + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + if (tenant_config.is_valid()) { + delay_recycle_scn.convert_for_tx(ObTimeUtil::current_time_ns() - + (tenant_config->_tx_result_retention * 1000L * 1000L * 1000L)); + } + if (delay_recycle_scn < tablet_recycle_scn) { + real_recycle_scn = delay_recycle_scn; + } else { + real_recycle_scn = tablet_recycle_scn; + } } return ret; } @@ -1028,7 +1039,7 @@ int ObTxTable::cleanout_tx_node(const transaction::ObTransID &tx_id, return ret; } -int ObTxTable::supplement_undo_actions_if_exist(ObTxData *&tx_data) +int ObTxTable::supplement_undo_actions_if_exist(ObTxData *tx_data) { return tx_data_table_.supplement_undo_actions_if_exist(tx_data); } diff --git a/src/storage/tx_table/ob_tx_table.h b/src/storage/tx_table/ob_tx_table.h index 07d76ed1a..631d237da 100644 --- a/src/storage/tx_table/ob_tx_table.h +++ b/src/storage/tx_table/ob_tx_table.h @@ -37,6 +37,9 @@ namespace storage class ObLS; class ObTxTable { + // Delay recycle tx data 5 minutes + const static int64_t TX_DATA_DELAY_RECYCLE_TIME_NS = 5L * 60L * 1000L * 1000L * 1000L; + public: static const int64_t INVALID_READ_EPOCH = -1; static const int64_t CHECK_AND_ONLINE_PRINT_INVERVAL_US = 5 * 1000 * 1000; // 5 seconds @@ -67,7 +70,7 @@ public: prepare_online_ts_(0), state_(OFFLINE), ls_(nullptr), - tx_data_table_(tx_data_table) {} + tx_data_table_(tx_data_table) {} ~ObTxTable() {} int init(ObLS *ls); @@ -89,18 +92,15 @@ public: // memory of tx_data needs to be allocated by this function // // @param [out] tx_data, a tx data allocated by slice allocator - int alloc_tx_data(ObTxData *&tx_data); + int alloc_tx_data(ObTxDataGuard &tx_data_guard); - void free_tx_data(ObTxData *tx_data) { tx_data_table_.free_tx_data(tx_data); } - - int deep_copy_tx_data(ObTxData *in_tx_data, ObTxData *&out_tx_data); + int deep_copy_tx_data(const ObTxDataGuard &in_tx_data_guard, ObTxDataGuard &out_tx_data_guard); // insert a tx data to tx data memtable // // @param [in] tx_data, which to be inserted int insert(ObTxData *&tx_data); - // =============== Interface for sstable to get txn information ===================== /** @@ -228,7 +228,7 @@ public: * * @param[in & out] tx_data The pointer of tx data to be supplemented which is in tx ctx. */ - int supplement_undo_actions_if_exist(ObTxData *&tx_data); + int supplement_undo_actions_if_exist(ObTxData *tx_data); int prepare_for_safe_destroy(); diff --git a/src/storage/tx_table/ob_tx_table_define.cpp b/src/storage/tx_table/ob_tx_table_define.cpp index 741548af1..a0b127184 100644 --- a/src/storage/tx_table/ob_tx_table_define.cpp +++ b/src/storage/tx_table/ob_tx_table_define.cpp @@ -271,7 +271,7 @@ int64_t ObTxCtxTableMeta::get_serialize_size_() const return len; } -DEF_TO_STRING(ObCommitSCNsArray::Node) +DEF_TO_STRING(ObCommitVersionsArray::Node) { int64_t pos = 0; J_KV(K_(start_scn), @@ -279,20 +279,22 @@ DEF_TO_STRING(ObCommitSCNsArray::Node) return pos; } -DEF_TO_STRING(ObCommitSCNsArray) +DEF_TO_STRING(ObCommitVersionsArray) { int64_t pos = 0; J_KV(KP(this), K(array_.count())); return pos; } -int ObCommitSCNsArray::serialize(char *buf, const int64_t buf_len, int64_t &pos) const +int ObCommitVersionsArray::serialize(char *buf, const int64_t buf_len, int64_t &pos) const { int ret = OB_SUCCESS; + const int64_t len = get_serialize_size_(); + if (OB_UNLIKELY(OB_ISNULL(buf) || buf_len <= 0 || pos > buf_len)) { ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "serialize ObCommitSCNsArray failed.", KR(ret), KP(buf), K(buf_len), + STORAGE_LOG(WARN, "serialize ObCommitVersionsArray failed.", KR(ret), KP(buf), K(buf_len), K(pos)); } else if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, UNIS_VERSION))) { STORAGE_LOG(WARN, "encode UNIS_VERSION failed.", KR(ret), KP(buf), K(buf_len), K(pos)); @@ -300,26 +302,30 @@ int ObCommitSCNsArray::serialize(char *buf, const int64_t buf_len, int64_t &pos) STORAGE_LOG(WARN, "encode length of commit versions array failed.", KR(ret), KP(buf), K(buf_len), K(pos)); } else if (OB_FAIL(serialize_(buf, buf_len, pos))) { - STORAGE_LOG(WARN, "serialize_ ObCommitSCNsArray failed.", KR(ret), KP(buf), K(buf_len), + STORAGE_LOG(WARN, "serialize_ ObCommitVersionsArray failed.", KR(ret), KP(buf), K(buf_len), K(pos)); } return ret; } -int ObCommitSCNsArray::deserialize(const char *buf, const int64_t data_len, int64_t &pos) +int ObCommitVersionsArray::deserialize(const char *buf, const int64_t data_len, int64_t &pos) { int ret = OB_SUCCESS; int64_t version = 0; int64_t len = 0; + array_.reuse(); - if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &version))) { + if (OB_UNLIKELY(nullptr == buf || data_len <= 0 || pos > data_len)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments.", KP(buf), K(data_len), K(ret)); + } else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &version))) { STORAGE_LOG(WARN, "decode version fail", K(version), K(data_len), K(pos), K(ret)); - } else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &len))) { - STORAGE_LOG(WARN, "decode data len fail", K(len), K(data_len), K(pos), K(ret)); } else if (version != UNIS_VERSION) { ret = OB_VERSION_NOT_MATCH; STORAGE_LOG(WARN, "object version mismatch", K(ret), K(version)); + } else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &len))) { + STORAGE_LOG(WARN, "decode data len fail", K(len), K(data_len), K(pos), K(ret)); } else if (OB_UNLIKELY(len < 0)) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "can't decode object with negative length", KR(ret), K(len)); @@ -331,7 +337,7 @@ int ObCommitSCNsArray::deserialize(const char *buf, const int64_t data_len, int6 pos = 0; array_.reuse(); if (OB_FAIL(deserialize_(buf + original_pos, len, pos))) { - STORAGE_LOG(WARN, "deserialize_ ObCommitSCNsArray fail", K(len), K(pos), K(ret)); + STORAGE_LOG(WARN, "deserialize_ ObCommitVersionsArray fail", K(len), K(pos), K(ret)); } pos += original_pos; } @@ -339,7 +345,7 @@ int ObCommitSCNsArray::deserialize(const char *buf, const int64_t data_len, int6 return ret; } -int64_t ObCommitSCNsArray::get_serialize_size() const +int64_t ObCommitVersionsArray::get_serialize_size() const { int64_t data_len = get_serialize_size_(); int64_t len = 0; @@ -349,7 +355,7 @@ int64_t ObCommitSCNsArray::get_serialize_size() const return len; } -int ObCommitSCNsArray::serialize_(char *buf, const int64_t buf_len, int64_t &pos) const +int ObCommitVersionsArray::serialize_(char *buf, const int64_t buf_len, int64_t &pos) const { int ret = OB_SUCCESS; for (int i = 0; OB_SUCC(ret) && i < array_.count(); i++) { @@ -358,11 +364,11 @@ int ObCommitSCNsArray::serialize_(char *buf, const int64_t buf_len, int64_t &pos return ret; } -int ObCommitSCNsArray::deserialize_(const char *buf, const int64_t data_len, int64_t &pos) +int ObCommitVersionsArray::deserialize_(const char *buf, const int64_t data_len, int64_t &pos) { int ret = OB_SUCCESS; - ObCommitSCNsArray::Node node; + ObCommitVersionsArray::Node node; while (OB_SUCC(ret) && pos < data_len) { LST_DO_CODE(OB_UNIS_DECODE, node.start_scn_, node.commit_version_); array_.push_back(node); @@ -371,7 +377,7 @@ int ObCommitSCNsArray::deserialize_(const char *buf, const int64_t data_len, int return ret; } -int64_t ObCommitSCNsArray::get_serialize_size_() const +int64_t ObCommitVersionsArray::get_serialize_size_() const { int64_t len = 0; for (int i = 0; i < array_.count(); i++) { @@ -380,7 +386,7 @@ int64_t ObCommitSCNsArray::get_serialize_size_() const return len; } -bool ObCommitSCNsArray::is_valid() +bool ObCommitVersionsArray::is_valid() { bool bool_ret = true; for (int i = 0; i < array_.count() - 1; i++) { @@ -396,7 +402,6 @@ bool ObCommitSCNsArray::is_valid() return bool_ret; } - } // end namespace transaction } // end namespace oceanbase diff --git a/src/storage/tx_table/ob_tx_table_define.h b/src/storage/tx_table/ob_tx_table_define.h index 448830f9b..421e59dbd 100644 --- a/src/storage/tx_table/ob_tx_table_define.h +++ b/src/storage/tx_table/ob_tx_table_define.h @@ -29,6 +29,27 @@ const int64_t MAX_TX_CTX_TABLE_VALUE_LENGTH = OB_MAX_USER_ROW_LENGTH - MAX_TX_CTX_TABLE_ID_LENGTH - MAX_TX_CTX_TABLE_META_LENGTH; static_assert(MAX_TX_CTX_TABLE_VALUE_LENGTH > 0, "MAX_TX_CTX_TABLE_VALUE_LENGTH is not enough"); + +struct TxDataDefaultAllocator : public ObIAllocator { + void *alloc(const int64_t size) override { + common::ObMemAttr attr; + attr.tenant_id_ = MTL_ID(); + attr.label_ = "TxData"; + if (size <= 0) { + abort(); + } + return ob_malloc(size, attr); + } + void* alloc(const int64_t size, const ObMemAttr &attr) override { return ob_malloc(size, attr); } + void free(void *ptr) override { ob_free(ptr); } + static TxDataDefaultAllocator &get_default_allocator() { + static TxDataDefaultAllocator default_allocator; + return default_allocator; + } +}; + +#define DEFAULT_TX_DATA_ALLOCATOR TxDataDefaultAllocator::get_default_allocator() + struct ObTxCtxTableCommonHeader { public: @@ -190,7 +211,7 @@ public: VIRTUAL_TO_STRING_KV("ObITxDataCheckFunctor", "tx_table"); }; -class ObCommitSCNsArray +class ObCommitVersionsArray { private: const static int64_t UNIS_VERSION = 1; @@ -220,7 +241,7 @@ public: void reset() { array_.reset(); } - ObCommitSCNsArray &operator=(const ObCommitSCNsArray& rhs) + ObCommitVersionsArray &operator=(const ObCommitVersionsArray& rhs) { this->array_.reset(); for (int i = 0; i < rhs.array_.count(); i++) { @@ -236,7 +257,7 @@ public: bool is_valid(); - static void print_to_stderr(const ObCommitSCNsArray &commit_versions) + static void print_to_stderr(const ObCommitVersionsArray &commit_versions) { fprintf(stderr, "pre-process data for upper trans version calculation : "); for (int i = 0; i < commit_versions.array_.count(); i++) { @@ -268,16 +289,16 @@ public: : is_inited_(false), cache_version_(), lock_(common::ObLatchIds::TX_TABLE_LOCK), - commit_scns_() {} + commit_versions_() {} void reset() { is_inited_ = false; cache_version_.reset(); - commit_scns_.reset(); + commit_versions_.reset(); } - TO_STRING_KV(K_(is_inited), K_(cache_version), K_(commit_scns)); + TO_STRING_KV(K_(is_inited), K_(cache_version), K_(commit_versions)); public: bool is_inited_; @@ -287,7 +308,7 @@ public: mutable common::TCRWLock lock_; - ObCommitSCNsArray commit_scns_; + ObCommitVersionsArray commit_versions_; }; } // storage diff --git a/src/storage/tx_table/ob_tx_table_iterator.cpp b/src/storage/tx_table/ob_tx_table_iterator.cpp index 4ce82af3a..898d4e278 100644 --- a/src/storage/tx_table/ob_tx_table_iterator.cpp +++ b/src/storage/tx_table/ob_tx_table_iterator.cpp @@ -11,13 +11,20 @@ */ #include "storage/tx_table/ob_tx_table_iterator.h" + +#include "lib/ob_define.h" +#include "lib/ob_errno.h" +#include "lib/utility/ob_macro_utils.h" #include "lib/utility/serialization.h" +#include "storage/blocksstable/ob_datum_range.h" #include "storage/tablet/ob_table_store_util.h" #include "storage/tx/ob_trans_ctx_mgr.h" #include "storage/tx/ob_trans_part_ctx.h" #include "storage/tx_storage/ob_ls_service.h" #include "storage/tx_table/ob_tx_ctx_memtable.h" #include "storage/tx_table/ob_tx_table.h" +#include + namespace oceanbase { using namespace share; @@ -25,8 +32,6 @@ using namespace share; namespace storage { -int64_t ObTxDataMemtableScanIterator::PERIODICAL_SELECT_INTERVAL_NS = 1000LL * 1000LL * 1000LL; - #define SSTABLE_HIDDEN_COLUMN_CNT ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt() #define GENERATE_ACCESS_CONTEXT \ @@ -55,13 +60,113 @@ int64_t ObTxDataMemtableScanIterator::PERIODICAL_SELECT_INTERVAL_NS = 1000LL * 1 STORAGE_LOG(WARN, "assign store row key failed.", KR(ret)); \ } +/**************** ObTxDataMemtableScanIterator::TxData2DatumRowConverter ************************/ + +int ObTxDataMemtableScanIterator::TxData2DatumRowConverter::init(ObTxData *tx_data) +{ + int ret = OB_SUCCESS; + int64_t need_ = 0; + int64_t pos = 0; + reset(); + if (OB_ISNULL(tx_data_ = tx_data)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(ERROR, "tx data is null", KR(ret)); + } else if (INT64_MAX != tx_data->tx_id_.get_id()) {// normal tx data need local buffer to serialize + buffer_len_ = tx_data->get_serialize_size(); + if (nullptr == (serialize_buffer_ = (char *)DEFAULT_TX_DATA_ALLOCATOR.alloc(buffer_len_))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "fail to serialize tx data, cause buffer allocated failed", + KR(ret), K(*this)); + } else if (OB_FAIL(tx_data->serialize(serialize_buffer_, buffer_len_, pos))) { + STORAGE_LOG(WARN, "can not serialize tx data to buffer", KR(ret), K(*this)); + } + } + return ret; +} + +void ObTxDataMemtableScanIterator::TxData2DatumRowConverter::reset() +{ + buffer_len_ = 0; + if (OB_NOT_NULL(serialize_buffer_)) { + ob_free(serialize_buffer_); + serialize_buffer_ = nullptr; + } + tx_data_ = nullptr; + generate_size_ = 0; + datum_row_.reset(); +} + +int ObTxDataMemtableScanIterator + ::TxData2DatumRowConverter::generate_next_now(const blocksstable::ObDatumRow *&row) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(tx_data_)) { + ret = OB_ITER_END;// no tx data remained + /*****************[NEED REMOVE IN FUTURE]*****************/ + } else if (INT64_MAX == tx_data_->tx_id_.get_id() && + generate_size_ == 1) { + ret = OB_ITER_END;// fake tx data datum row has been generated + /*********************************************************/ + } else if (INT64_MAX != tx_data_->tx_id_.get_id() && + generate_size_ == std::ceil(buffer_len_ * 1.0 / common::OB_MAX_VARCHAR_LENGTH)) { + ret = OB_ITER_END;// all tx data datum row has been generated + } else { + if (generate_size_ >= 1) { + STORAGE_LOG(INFO, "meet big tx data", KR(ret), K(*this)); + } + datum_row_.reset(); + new (&datum_row_) ObDatumRow();// CAUTIONS: this is needed, or will core dump + if (OB_FAIL(datum_row_.init(DEFAULT_TX_DATA_ALLOCATOR, + TX_DATA_MAX_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT))) { + STORAGE_LOG(ERROR, "fail to init datum row", KR(ret), K(*this)); + } else { + datum_row_.row_flag_.set_flag(blocksstable::ObDmlFlag::DF_INSERT); + datum_row_.storage_datums_[TX_DATA_ID_COLUMN].set_int(tx_data_->tx_id_.get_id()); + datum_row_.storage_datums_[TX_DATA_IDX_COLUMN].set_int(generate_size_); + datum_row_.storage_datums_[TX_DATA_IDX_COLUMN + 1].set_int(-4096);// storage layer needed + datum_row_.storage_datums_[TX_DATA_IDX_COLUMN + 2].set_int(0);// storage layer needed + int64_t total_row_cnt_column = TX_DATA_TOTAL_ROW_CNT_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; + int64_t end_ts_column = TX_DATA_END_TS_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; + int64_t value_column = TX_DATA_VAL_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; + char *p_value_begin = serialize_buffer_ + common::OB_MAX_VARCHAR_LENGTH * generate_size_; + generate_size_++; + ObString value; + /*****************[NEED REMOVE IN FUTURE]*****************/ + // TODO : remove this after the sstables do not need upper trans version + if (INT64_MAX == tx_data_->tx_id_.get_id()) { + // NOTE : this fake tx data is generated in + // ObTxDataMemtable::pre_process_commit_version_row_ + datum_row_.storage_datums_[total_row_cnt_column].set_int(1); + datum_row_.storage_datums_[end_ts_column].set_int(INT64_MAX); + value.assign((char *)(tx_data_->start_scn_.get_val_for_tx()), tx_data_->commit_version_.get_val_for_tx()); + /*********************************************************/ + } else { + datum_row_.storage_datums_[total_row_cnt_column].set_int(std::ceil(buffer_len_ * 1.0 / common::OB_MAX_VARCHAR_LENGTH)); + datum_row_.storage_datums_[end_ts_column].set_int(tx_data_->end_scn_.get_val_for_tx()); + value.assign(p_value_begin, + std::min(common::OB_MAX_VARCHAR_LENGTH, + buffer_len_ - (p_value_begin - serialize_buffer_))); + } + datum_row_.storage_datums_[value_column].set_string(value); + datum_row_.set_first_multi_version_row();// storage layer needed for compatibility + datum_row_.set_last_multi_version_row();// storage layer needed for compatibility + datum_row_.set_compacted_multi_version_row();// storage layer needed for compatibility + row = &datum_row_; + } + } + return ret; +} + +/******************** ObTxDataMemtableScanIterator::TxData2DatumRowConverter **********************/ + +/***************************** ObTxDataMemtableScanIterator **********************************/ + int ObTxDataMemtableScanIterator::init(ObTxDataMemtable *tx_data_memtable) { int ret = OB_SUCCESS; if (IS_INIT) { reset(); } - if (OB_ISNULL(tx_data_memtable)) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "init ObTxDataMemtableScanIterator with a null tx_data_memtable.", KR(ret)); @@ -82,24 +187,15 @@ int ObTxDataMemtableScanIterator::init(ObTxDataMemtable *tx_data_memtable) K(inserted_count), K(deleted_count), KPC(tx_data_memtable)); - } else if (OB_FAIL(tx_data_memtable->prepare_tx_data_list())) { - STORAGE_LOG(WARN, "prepare tx data list failed.", KR(ret)); - } else if (OB_ISNULL(cur_node_ = tx_data_memtable->get_sorted_list_head())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error.", KR(ret), KPC(tx_data_memtable)); - } else if (OB_FAIL(row_.init(arena_allocator_, TX_DATA_MAX_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT))) { - STORAGE_LOG(WARN, "Failed to init datum row", KR(ret)); - } else if (OB_FAIL(buf_.reserve(BUF_LENGTH))) { - STORAGE_LOG(WARN, "Failed to reserve tx local buffer", K(ret)); + } else if (OB_FAIL(init_iterate_range_(tx_data_memtable))) { + STORAGE_LOG(WARN, "init iterate range failed.", KR(ret)); + } else if (OB_FAIL(init_iterate_count_(tx_data_memtable))) { + STORAGE_LOG(WARN, "init iterate count failed.", KR(ret)); } else { - row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT); - // cur_node_ point to the next tx data - cur_node_ = cur_node_->next_; tx_data_memtable_ = tx_data_memtable; - cur_max_commit_version_.set_min(); - pre_start_scn_.set_min(); - tx_data_row_cnt_ = 0; + iterate_row_cnt_ = 0; pre_tx_data_ = nullptr; + DEBUG_drop_tx_data_cnt_ = 0; is_inited_ = true; } @@ -107,68 +203,178 @@ int ObTxDataMemtableScanIterator::init(ObTxDataMemtable *tx_data_memtable) return ret; } +int ObTxDataMemtableScanIterator::init_iterate_range_(ObTxDataMemtable *tx_data_memtable) +{ + int ret = OB_SUCCESS; + + const ObObj *start_obj = nullptr; + const ObObj *end_obj = nullptr; + + // get start tx id + if (range_.get_start_key().is_min_rowkey()) { + start_tx_id_ = 0; + } else if (OB_ISNULL(start_obj + = range_.get_start_key().get_store_rowkey().get_rowkey().get_obj_ptr())) { + STORAGE_LOG(WARN, "get start obj from range failed.", KR(ret), K(range_)); + } else if (OB_FAIL(start_obj[0].get_int(start_tx_id_))) { + STORAGE_LOG(WARN, "get start tx id from start obj failed", KR(ret), KPC(start_obj)); + } else if (start_tx_id_ < 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "get an invalid start tx id from start obj ", KR(ret), KPC(start_obj)); + } + + // set iterate start node by start tx id + if (OB_FAIL(ret)) { + } else if (0 == start_tx_id_) { + // iterate from sort list head + if (OB_ISNULL(cur_node_ = tx_data_memtable->get_sorted_list_head())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected error.", KR(ret), KPC(tx_data_memtable)); + } + } else { + // iterate from range start + ObTxDataGuard tx_data_guard; + tx_data_guard.reset(); + ObTxData *tx_data = nullptr; + if (OB_FAIL(tx_data_memtable->get_tx_data(start_tx_id_, tx_data_guard))) { + STORAGE_LOG(WARN, "get tx data from tx data memtable failed.", KR(ret), K(start_tx_id_), + KPC(tx_data_memtable)); + } else if (OB_ISNULL(tx_data = tx_data_guard.tx_data())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "tx data is unexpected nullptr in tx data guard", KR(ret), + KPC(tx_data_memtable)); + } else { + cur_node_ = &tx_data->sort_list_node_; + } + } + + // set end tx id + if (OB_FAIL(ret)) { + } else if (range_.get_end_key().is_max_rowkey()) { + end_tx_id_ = INT64_MAX; + } else if (OB_ISNULL(end_obj + = range_.get_end_key().get_store_rowkey().get_rowkey().get_obj_ptr())) { + STORAGE_LOG(WARN, "get end obj from range failed.", KR(ret), K(range_)); + } else if (OB_FAIL(end_obj[0].get_int(end_tx_id_))) { + STORAGE_LOG(WARN, "get end tx id from end obj failed", KR(ret), KPC(end_obj)); + } + + return ret; +} + +int ObTxDataMemtableScanIterator::init_iterate_count_(ObTxDataMemtable *tx_data_memtable) +{ + int ret = OB_SUCCESS; + if (range_.get_start_key().is_min_rowkey() && range_.get_end_key().is_max_rowkey()) { + row_cnt_to_dump_ = tx_data_memtable->get_tx_data_count(); + } else if (OB_FAIL(tx_data_memtable->get_tx_data_cnt_by_tx_id(start_tx_id_, row_cnt_to_dump_))) { + STORAGE_LOG(WARN, "get tx data count by tx id failed.", KR(ret)); + } else { + } + return ret; +} + void ObTxDataMemtableScanIterator::reset() { - if (OB_NOT_NULL(tx_data_memtable_)) { - tx_data_memtable_->reset_is_iterating(); - } - dump_tx_data_done_ = false; - cur_max_commit_version_.set_min(); - pre_start_scn_.set_min(); - tx_data_row_cnt_ = 0; + iterate_row_cnt_ = 0; + start_tx_id_ = 0; + end_tx_id_ = INT64_MAX; + row_cnt_to_dump_ = 0; pre_tx_data_ = nullptr; - arena_allocator_.reset(); cur_node_ = nullptr; - row_.reset(); - buf_.reset(); tx_data_memtable_ = nullptr; is_inited_ = false; + DEBUG_drop_tx_data_cnt_ = 0; } void ObTxDataMemtableScanIterator::reuse() { reset(); } -int ObTxDataMemtableScanIterator::inner_get_next_row(const ObDatumRow *&row) +int ObTxDataMemtableScanIterator::get_next_tx_data_(ObTxData *&tx_data) +{ + int ret = OB_SUCCESS; + + if (OB_ISNULL(cur_node_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "current node is unexpected nullptr.", KR(ret), KPC(tx_data_memtable_)); + } else if (OB_ISNULL(cur_node_->next_)) { + ret = OB_ITER_END; + } else if (FALSE_IT(cur_node_ = &(cur_node_->next_->sort_list_node_))) { + } else if (FALSE_IT(tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node_))) { + } else if (OB_FAIL(drop_and_get_tx_data_(tx_data))) { // handle savepoint rollback tx data + STORAGE_LOG(WARN, "drop and get tx data failed", KR(ret)); + } else if (OB_NOT_NULL(pre_tx_data_) && tx_data->tx_id_ <= pre_tx_data_->tx_id_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "iterate an invalid rowkey in a single tx data memtable", KR(ret), + KPC(pre_tx_data_), KPC(tx_data), KPC(tx_data_memtable_)); + } else if (FALSE_IT(pre_tx_data_ = tx_data)) { + } else if (tx_data->tx_id_ <= start_tx_id_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "iterate an out of range row key", KR(ret), KPC(tx_data)); + } else if (tx_data->tx_id_.get_id() > end_tx_id_) { + ret = OB_ITER_END; + } else { + // find a tx data which need to be flushed + } + + return ret; +} + +int ObTxDataMemtableScanIterator::drop_and_get_tx_data_(ObTxData *&tx_data) +{ + int ret = OB_SUCCESS; + while (OB_NOT_NULL(cur_node_->next_)) { + ObTxData *next_tx_data = cur_node_->next_; + + // the tx datas having the same rowkey must be rollback tx data excpet one commit tx data + if (OB_UNLIKELY(next_tx_data->tx_id_ == tx_data->tx_id_)) { + cur_node_ = &(cur_node_->next_->sort_list_node_); + row_cnt_to_dump_--; + DEBUG_drop_tx_data_cnt_++; + if (OB_UNLIKELY(next_tx_data->end_scn_ > tx_data->end_scn_)) { + // pointer to next_tx_data cause its end_log_ts is larger + STORAGE_LOG(DEBUG, "drop one rollback tx data", "droped : ", to_cstring(tx_data), "keeped", to_cstring(next_tx_data)); + tx_data = next_tx_data; + } else { + STORAGE_LOG(DEBUG, "drop one rollback tx data", "droped : ", to_cstring(next_tx_data), "keeped", to_cstring(tx_data)); + } + } else { + break; + } + } + return ret; +} + +int ObTxDataMemtableScanIterator::inner_get_next_row(const blocksstable::ObDatumRow *&row) { int ret = OB_SUCCESS; row = nullptr; + ObTxData *tx_data = nullptr; if (IS_NOT_INIT) { ret = OB_NOT_INIT; STORAGE_LOG(WARN, "tx data memtable scan iterator is not inited"); - } else if (OB_NOT_NULL(cur_node_)) { - ret = get_next_tx_data_row_(row); - } else if (!dump_tx_data_done_) { - // cur_node == nullptr && dump_tx_data_done == false - // which means this row should be commit version row - dump_tx_data_done_ = true; - if (tx_data_row_cnt_ != tx_data_memtable_->get_inserted_count() - tx_data_memtable_->get_deleted_count()) { - ret = OB_ERR_UNEXPECTED; - int64_t tx_data_count_in_memtable = tx_data_memtable_->get_inserted_count(); - STORAGE_LOG(ERROR, "iterate tx data row count is not equal to tx data in memtable", KR(ret), - K(tx_data_row_cnt_), K(tx_data_count_in_memtable), KPC(tx_data_memtable_)); - } else if (OB_FAIL(prepare_commit_scn_list_())) { - STORAGE_LOG(WARN, "prepare commit version array for calculating upper_trans_version failed.", - KR(ret), KPC(tx_data_memtable_)); - } else { - cur_node_ = tx_data_memtable_->get_sorted_list_head(); - if (OB_ISNULL(cur_node_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "current node unexpected null.", KR(ret), KPC(cur_node_), - KPC(tx_data_memtable_)); - } else if (OB_ISNULL(cur_node_ = cur_node_->next_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "current node unexpected null.", KR(ret), KPC(cur_node_), - KPC(tx_data_memtable_)); - } else if (OB_FAIL(get_next_commit_scn_row_(row))) { - STORAGE_LOG(WARN, "get pre-process commit versions row failed.", KR(ret), KPC(row)); - } else { - STORAGE_LOG(INFO, "successfully get next commit versions row!", KPC(row)); + } else if (OB_SUCC(tx_data_2_datum_converter_.generate_next_now(row))) { + // do nothing, next row is assigned out + } else if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to generate datum row", KR(ret), K_(tx_data_2_datum_converter)); + } else {// no lefeted row in tx_data_2_datum_converter + if (OB_FAIL(get_next_tx_data_(tx_data))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "get next tx data failed.", KR(ret), KPC(tx_data_memtable_)); } + } else if (OB_ISNULL(tx_data)) { + ret = OB_BAD_NULL_ERROR; + STORAGE_LOG(ERROR, "tx data is nullptr", KR(ret), KPC(tx_data_memtable_)); + } else if (OB_FAIL(tx_data_2_datum_converter_.init(tx_data))) { + STORAGE_LOG(WARN, "fail to convert tx data to datum", KR(ret), KPC(tx_data_memtable_)); + } else if (OB_FAIL(tx_data_2_datum_converter_.generate_next_now(row))) { + STORAGE_LOG(WARN, "fail to get row from tx_data_2_datum_converter", + KR(ret), KPC(tx_data_memtable_), K_(tx_data_2_datum_converter)); + } else if (++iterate_row_cnt_ > row_cnt_to_dump_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "invalid iterate row count", + KR(ret), K(iterate_row_cnt_), K(row_cnt_to_dump_)); } - } else { - // cur_node == nullptr && dump_tx_data_done == true - // which means dump tx data sstable done - ret = OB_ITER_END; } if (OB_NOT_NULL(row) @@ -176,522 +382,21 @@ int ObTxDataMemtableScanIterator::inner_get_next_row(const ObDatumRow *&row) ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "Invalid tx data sstable row", KPC(row)); } - return ret; -} - -int ObTxDataMemtableScanIterator::prepare_commit_scn_list_() -{ - int ret = tx_data_memtable_->prepare_commit_scn_list(); - return ret; -} - -int ObTxDataMemtableScanIterator::get_next_tx_data_row_(const blocksstable::ObDatumRow *&row) -{ - int ret = OB_SUCCESS; - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node_); - int64_t pos = 0; - - // TODO : @gengli - // if there are too many undo actions, the serialize_size can be very large - int64_t serialize_size = tx_data->get_serialize_size(); - if (OB_NOT_NULL(pre_tx_data_) && tx_data->tx_id_ <= pre_tx_data_->tx_id_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "iterate the same rowkey in a single tx data memtable", KR(ret), - KPC(pre_tx_data_), KPC(tx_data), KPC(tx_data_memtable_)); - } else if (serialize_size > common::OB_MAX_VARCHAR_LENGTH) { - ret = OB_SIZE_OVERFLOW; - STORAGE_LOG(WARN, "Too Much Undo Actions", KR(ret), KPC(tx_data)); - } else if (OB_FAIL(buf_.reserve(serialize_size))) { - STORAGE_LOG(WARN, "Failed to reserve local buffer", KR(ret), KPC(tx_data)); - } else if (OB_FAIL(tx_data->serialize(buf_.get_ptr(), serialize_size, pos))) { - STORAGE_LOG(WARN, "failed to serialize tx state info", KR(ret), KPC(tx_data), K(pos)); - } else { - row_.storage_datums_[TX_DATA_ID_COLUMN].set_int(tx_data->tx_id_.get_id()); - row_.storage_datums_[TX_DATA_IDX_COLUMN].set_int(0); - row_.storage_datums_[TX_DATA_IDX_COLUMN + 1].set_int(-4096); - row_.storage_datums_[TX_DATA_IDX_COLUMN + 2].set_int(0); - - int64_t total_row_cnt_column = TX_DATA_TOTAL_ROW_CNT_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; - int64_t end_ts_column = TX_DATA_END_TS_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; - int64_t value_column = TX_DATA_VAL_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; - row_.storage_datums_[total_row_cnt_column].set_int(1); - row_.storage_datums_[end_ts_column].set_int(tx_data->end_scn_.get_val_for_tx()); - row_.storage_datums_[value_column].set_string(ObString(serialize_size, buf_.get_ptr())); - row_.set_first_multi_version_row(); - row_.set_last_multi_version_row(); - row_.set_compacted_multi_version_row(); - row = &row_; - - pre_tx_data_ = tx_data; - ATOMIC_INC(&tx_data_row_cnt_); - - // fill in a new row successfully - // point to the next tx data - cur_node_ = cur_node_->next_; - } - return ret; -} - -// This function is called after sorting tx_data by start_scn and the following steps is -// executed: -// 1. Select (start_scn, commit_version) point per second and push them into an array. -// 2. Read (start_scn, commit_version) array from the latest tx data sstable. -// 3. Get the recycle_scn to filtrate the point which is not needed any more. -// 4. Merge the arrays above. This procedure should filtrate the points are not needed and keep the -// commit versions monotonically increasing. -// 5. Serialize the merged array into one sstable row. -int ObTxDataMemtableScanIterator::get_next_commit_scn_row_(const blocksstable::ObDatumRow *&row) -{ - int ret = OB_SUCCESS; - SCN recycle_scn = SCN::min_scn(); - int64_t serialize_size = 0; - ObCommitSCNsArray cur_commit_scns; - ObCommitSCNsArray past_commit_scns; - ObCommitSCNsArray merged_commit_scns; - - if (OB_FAIL(fill_in_cur_commit_scns_(cur_commit_scns) /*step 1*/)) { - STORAGE_LOG(WARN, "periodical select commit version failed.", KR(ret)); - } else if (tx_data_row_cnt_ != DEBUG_iter_commit_ts_cnt_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected iter count when pre process commit versions array.", K(tx_data_row_cnt_), K(DEBUG_iter_commit_ts_cnt_)); - DEBUG_print_start_scn_list_(); - ob_abort(); - } else if (OB_FAIL(get_past_commit_scns_(past_commit_scns) /*step 2*/)) { - STORAGE_LOG(WARN, "get past commit versions failed.", KR(ret)); - } else if (OB_FAIL( - tx_data_memtable_->get_tx_data_memtable_mgr()->get_tx_data_table()->get_recycle_scn( - recycle_scn) /*step 3*/)) { - STORAGE_LOG(WARN, "get recycle ts failed.", KR(ret)); - } else if (OB_FAIL(merge_cur_and_past_commit_verisons_(recycle_scn, cur_commit_scns, - past_commit_scns, - merged_commit_scns) /*step 4*/)) { - STORAGE_LOG(WARN, "merge current and past commit versions failed.", KR(ret)); - } else if (!merged_commit_scns.is_valid()) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "invalid commit versions", KR(ret)); - } else if (OB_FAIL(DEBUG_try_calc_upper_and_check_(merged_commit_scns))) { - } else if (OB_FAIL( - set_row_with_merged_commit_scns_(merged_commit_scns, row) /*step 5*/)) { - STORAGE_LOG(WARN, "set row with merged commit versions failed.", KR(ret)); - } else { - // get commit version row succeed. - } - - return ret; -} - -int ObTxDataMemtableScanIterator::DEBUG_try_calc_upper_and_check_(ObCommitSCNsArray &merged_commit_versions) -{ - int ret = OB_SUCCESS; - - ObTxDataSortListNode *cur_node = tx_data_memtable_->get_sorted_list_head()->next_; - int64_t DEBUG_iter_cnt = 0; - while (OB_SUCC(ret) && OB_NOT_NULL(cur_node)) { - DEBUG_iter_cnt++; - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); - cur_node = cur_node->next_; - - if (ObTxData::COMMIT != tx_data->state_) { - continue; - } - - SCN upper_trans_version = SCN::min_scn(); - if (OB_FAIL(DEBUG_fake_calc_upper_trans_version(tx_data->start_scn_, upper_trans_version, merged_commit_versions))) { - STORAGE_LOG(ERROR, "invalid upper trans version", KR(ret)); - } else if (upper_trans_version < tx_data->commit_version_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "invalid upper trans version", KR(ret), K(upper_trans_version), KPC(tx_data)); - } - - if (OB_FAIL(ret)) { - DEBUG_print_start_scn_list_(); - DEBUG_print_merged_commit_versions_(merged_commit_versions); - } - } - if (OB_SUCC(ret) && DEBUG_iter_cnt != tx_data_row_cnt_) { - ret = OB_SUCCESS; - STORAGE_LOG(ERROR, "invalid iter cnt", KR(ret), K(DEBUG_iter_cnt), K(tx_data_row_cnt_)); - } - - return ret; -} - -int ObTxDataMemtableScanIterator::DEBUG_fake_calc_upper_trans_version(const SCN sstable_end_scn, - SCN &upper_trans_version, - ObCommitSCNsArray &merged_commit_versions) -{ - int ret = OB_SUCCESS; - - ObIArray &array = merged_commit_versions.array_; - int l = 0; - int r = array.count() - 1; - - // Binary find the first start_scn that is greater than or equal to sstable_end_scn - while (l < r) { - int mid = (l + r) >> 1; - if (array.at(mid).start_scn_ < sstable_end_scn) { - l = mid + 1; - } else { - r = mid; - } - } - - // Check if the start_scn is greater than or equal to the sstable_end_scn. If not, delay the - // upper_trans_version calculation to the next time. - if (0 == array.count() || !array.at(l).commit_version_.is_valid()) { - upper_trans_version.set_max(); - ret = OB_ERR_UNDEFINED; - STORAGE_LOG(WARN, "unexpected array count or commit version", K(array.count()), K(array.at(l))); - } else { - upper_trans_version = array.at(l).commit_version_; - } - - return ret; -} - -void ObTxDataMemtableScanIterator::DEBUG_print_start_scn_list_() -{ - int ret = OB_SUCCESS; - const char *real_fname = "tx_data_start_scn_list"; - FILE *fd = NULL; - - if (NULL == (fd = fopen(real_fname, "w"))) { - ret = OB_IO_ERROR; - STORAGE_LOG(WARN, "open file fail:", K(real_fname)); - } else { - auto tenant_id = MTL_ID(); - fprintf(fd, "tenant_id=%ld \n", tenant_id); - ObTxDataSortListNode *cur_node = tx_data_memtable_->get_sorted_list_head()->next_; - while (OB_NOT_NULL(cur_node)) { - ObTxData *tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node); - cur_node = cur_node->next_; - - fprintf(fd, - "ObTxData : tx_id=%-19ld is_in_memtable=%-3d state=%-8s start_scn=%-19s " - "end_scn=%-19s " - "commit_version=%-19s\n", - tx_data->tx_id_.get_id(), - tx_data->is_in_tx_data_table_, - ObTxData::get_state_string(tx_data->state_), - to_cstring(tx_data->start_scn_), - to_cstring(tx_data->end_scn_), - to_cstring(tx_data->commit_version_)); - } - } - - if (NULL != fd) { - fprintf(fd, "end of start scn list\n"); - fclose(fd); - fd = NULL; - } - if (OB_FAIL(ret)) { - STORAGE_LOG(WARN, "dump start scn list fail", K(real_fname), K(ret)); - } -} - -void ObTxDataMemtableScanIterator::DEBUG_print_merged_commit_versions_(ObCommitSCNsArray &merged_commit_versions) -{ - int ret = OB_SUCCESS; - const auto &array = merged_commit_versions.array_; - const char *real_fname = "merge_commit_versions"; - FILE *fd = NULL; - - if (NULL == (fd = fopen(real_fname, "w"))) { - ret = OB_IO_ERROR; - STORAGE_LOG(WARN, "open file fail:", K(real_fname)); - } else { - auto tenant_id = MTL_ID(); - fprintf(fd, "tenant_id=%ld \n", tenant_id); - for (int i = 0; i < array.count(); i++) { - fprintf(fd, - "start_scn=%-19s " - "commit_version=%-19s\n", - to_cstring(array.at(i).start_scn_), - to_cstring(array.at(i).commit_version_)); - } - } - - if (NULL != fd) { - fprintf(fd, "end of commit versions array\n"); - fclose(fd); - fd = NULL; - } - if (OB_FAIL(ret)) { - STORAGE_LOG(WARN, "dump commit versions fail", K(real_fname), K(ret)); - } -} - - -int ObTxDataMemtableScanIterator::DEBUG_check_past_and_cur_arr( - ObCommitSCNsArray &cur_commit_scns, ObCommitSCNsArray &past_commit_scns) -{ - int ret = OB_SUCCESS; - auto &cur_arr = cur_commit_scns.array_; - auto &past_arr = past_commit_scns.array_; - STORAGE_LOG(INFO, "start debug check past and cur array", K(cur_arr.count()), - K(past_arr.count())); - - for (int i = 0; OB_SUCC(ret) && i < cur_arr.count() - 1; i++) { - if (cur_arr.at(i).start_scn_ > cur_arr.at(i + 1).start_scn_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected error in cur_arr", K(cur_arr.at(i)), K(cur_arr.at(i + 1))); - } - } - - for (int i = 0; OB_SUCC(ret) && i < past_arr.count() - 1; i++) { - if (past_arr.at(i).start_scn_ > past_arr.at(i + 1).start_scn_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected error in cur_arr", K(past_arr.at(i)), K(past_arr.at(i + 1))); - } - } - - STORAGE_LOG(INFO, "finish debug check past and cur array", KR(ret), K(cur_arr.count()), - K(past_arr.count())); - return ret; -} - -int ObTxDataMemtableScanIterator::fill_in_cur_commit_scns_(ObCommitSCNsArray &cur_commit_scns) -{ - int ret = OB_SUCCESS; - ObCommitSCNsArray::Node node; - DEBUG_iter_commit_ts_cnt_ = 0; - DEBUG_last_start_scn_.set_min(); - while (OB_SUCC(periodical_get_next_commit_scn_(node))) { - cur_commit_scns.array_.push_back(node); - } if (OB_ITER_END == ret) { - ret = OB_SUCCESS; - } else { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "Unexpected error occurs when periodical select commit version", KR(ret), - KPC(tx_data_memtable_)); - } - - return ret; -} - -int ObTxDataMemtableScanIterator::periodical_get_next_commit_scn_(ObCommitSCNsArray::Node &node) -{ - int ret = OB_SUCCESS; - ObTxData *tx_data = nullptr; - - while (OB_SUCC(ret) && nullptr != cur_node_) { - ObTxData *tmp_tx_data = ObTxData::get_tx_data_by_sort_list_node(cur_node_); - cur_node_ = cur_node_->next_; - DEBUG_iter_commit_ts_cnt_++; - - // avoid rollback or abort transaction influencing commit versions array - if (ObTxData::COMMIT != tmp_tx_data->state_) { - continue; + if (iterate_row_cnt_ != row_cnt_to_dump_) { + STORAGE_LOG(ERROR, "invalid iterate row count", + K(iterate_row_cnt_), K(row_cnt_to_dump_), KPC(tx_data_memtable_)); } else { - tx_data = tmp_tx_data; - } - - if (DEBUG_last_start_scn_ > tx_data->start_scn_) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected start scn order", K(DEBUG_last_start_scn_), KPC(tx_data)); - break; - } else { - DEBUG_last_start_scn_ = tx_data->start_scn_; - } - - // update pre_commit_version - if (tx_data->commit_version_ > cur_max_commit_version_) { - cur_max_commit_version_ = tx_data->commit_version_; - } - - // If this tx data is the first tx data in sorted list or its start_scn is 1_s larger than - // the pre_start_scn, we use this start_scn to calculate upper_trans_version - if (pre_start_scn_.is_min() || - tx_data->start_scn_ >= SCN::plus(pre_start_scn_, PERIODICAL_SELECT_INTERVAL_NS)/*1s*/) { - pre_start_scn_ = tx_data->start_scn_; - break; + STORAGE_LOG(INFO, + "iterate tx data memtable done.", + K(iterate_row_cnt_), + K(row_cnt_to_dump_), + K(start_tx_id_), + K(end_tx_id_), + KPC(tx_data_memtable_)); } } - - if (nullptr != tx_data) { - node.start_scn_ = tx_data->start_scn_; - // use cur_max_commit_version_ to keep the commit versions monotonically increasing - node.commit_version_ = cur_max_commit_version_; - tx_data = nullptr; - } else if (nullptr == cur_node_) { - ret = OB_ITER_END; - } - - return ret; -} - -int ObTxDataMemtableScanIterator::get_past_commit_scns_( - ObCommitSCNsArray &past_commit_scns) -{ - int ret = OB_SUCCESS; - ObLSTabletService *tablet_svr - = tx_data_memtable_->get_tx_data_memtable_mgr()->get_ls_tablet_svr(); - ObTableIterParam iter_param = iter_param_; - ObTabletHandle &tablet_handle = iter_param.tablet_handle_; - - if (tablet_handle.is_valid()) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "tablet handle should be empty", KR(ret), K(tablet_handle)); - } else if (OB_ISNULL(tablet_svr)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "tablet svr is nullptr", KR(ret), KPC(tx_data_memtable_)); - } else if (OB_FAIL(tablet_svr->get_tablet(LS_TX_DATA_TABLET, tablet_handle))) { - STORAGE_LOG(WARN, "get tablet from ls tablet service failed.", KR(ret)); - } else if (OB_UNLIKELY(!tablet_handle.is_valid())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "invalid tablet handle", KR(ret), K(tablet_handle)); - } else { - // get the lastest sstable - ObITable *table - = tablet_handle.get_obj()->get_table_store().get_minor_sstables().get_boundary_table( - true /*is_last*/); - - if (OB_NOT_NULL(table)) { - ObCommitVersionsGetter getter(iter_param, table); - if (OB_FAIL(getter.get_next_row(past_commit_scns))) { - STORAGE_LOG(WARN, "get commit versions from tx data sstable failed.", KR(ret)); - } - } else { - STORAGE_LOG(DEBUG, "There is no tx data sstable yet", KR(ret), KPC(table)); - } - } - - return ret; -} - -int ObTxDataMemtableScanIterator::merge_cur_and_past_commit_verisons_(const SCN recycle_scn, - ObCommitSCNsArray &cur_commit_scns, - ObCommitSCNsArray &past_commit_scns, - ObCommitSCNsArray &merged_commit_scns) -{ - int ret = OB_SUCCESS; - ObIArray &cur_arr = cur_commit_scns.array_; - ObIArray &past_arr = past_commit_scns.array_; - ObIArray &merged_arr = merged_commit_scns.array_; - - - int64_t cur_size = cur_commit_scns.get_serialize_size(); - int64_t past_size = past_commit_scns.get_serialize_size(); - int64_t step_len = 1; - if (cur_size + past_size > common::OB_MAX_VARCHAR_LENGTH) { - STORAGE_LOG(INFO, - "Too Much Pre-Process Data to Desirialize", - K(recycle_scn), - K(past_size), - K(cur_size), - "past_array_count", past_commit_scns.array_.count(), - "cur_array_count", cur_commit_scns.array_.count()); - step_len = step_len + ((cur_size + past_size) / OB_MAX_VARCHAR_LENGTH); - } - - // here we merge the past commit versions and current commit versions. To keep merged array correct, the node in past - // array whose start_scn is larger than the minimum start_scn in current array will be dropped. The reason is in this - // issue: https://work.aone.alibaba-inc.com/issue/43389863 - SCN cur_min_start_scn = cur_arr.count() > 0 ? cur_arr.at(0).start_scn_ : SCN::max_scn(); - SCN max_commit_version = SCN::min_scn(); - if (OB_FAIL( - merge_pre_process_node_(step_len, cur_min_start_scn, recycle_scn, past_arr, max_commit_version, merged_arr))) { - STORAGE_LOG(WARN, "merge past commit versions failed.", KR(ret), K(past_arr), KPC(tx_data_memtable_)); - } else if (OB_FAIL( - merge_pre_process_node_(step_len, SCN::max_scn(), recycle_scn, cur_arr, max_commit_version, merged_arr))) { - STORAGE_LOG(WARN, "merge current commit versions failed.", KR(ret), K(cur_arr), KPC(tx_data_memtable_)); - } else if (0 == merged_arr.count()) { - if (OB_FAIL(merged_arr.push_back(ObCommitSCNsArray::Node(SCN::max_scn(), SCN::max_scn())))) { - STORAGE_LOG(WARN, "push back commit version node failed.", KR(ret), KPC(tx_data_memtable_)); - } else { - STORAGE_LOG(INFO, "push back an INT64_MAX node for upper trans version calculation", K(merged_arr)); - } - } - - STORAGE_LOG(INFO, - "generate commit versions array finish.", - K(recycle_scn), - K(step_len), - "past_array_count", past_commit_scns.array_.count(), - "cur_array_count", cur_commit_scns.array_.count(), - "merged_array_count", merged_commit_scns.array_.count()); - - return ret; -} - -int ObTxDataMemtableScanIterator::merge_pre_process_node_(const int64_t step_len, - const SCN start_scn_limit, - const SCN recycle_scn, - const ObIArray &data_arr, - SCN &max_commit_version, - ObIArray &merged_arr) -{ - int ret = OB_SUCCESS; - int64_t arr_len = data_arr.count(); - if (arr_len <= 0) { - // skip push back - } else { - // push back pre-process node except the last one - int64_t i = 0; - for (; OB_SUCC(ret) && i < arr_len - 1; i += step_len) { - if (data_arr.at(i).start_scn_ >= start_scn_limit) { - break; - } - max_commit_version = std::max(max_commit_version, data_arr.at(i).commit_version_); - ObCommitSCNsArray::Node new_node(data_arr.at(i).start_scn_, max_commit_version); - if (new_node.commit_version_ <= recycle_scn) { - // this tx data should be recycled - // do nothing - } else if (OB_FAIL(merged_arr.push_back(new_node))) { - STORAGE_LOG(WARN, "push back commit version node failed.", KR(ret), KPC(tx_data_memtable_)); - } - } - - // push back the last pre-process node - max_commit_version = std::max(max_commit_version, data_arr.at(arr_len - 1).commit_version_); - if (OB_SUCC(ret) && data_arr.at(arr_len - 1).start_scn_ < start_scn_limit) { - ObCommitSCNsArray::Node new_node(data_arr.at(arr_len - 1).start_scn_, max_commit_version); - if (OB_FAIL(merged_arr.push_back(new_node))) { - STORAGE_LOG(WARN, "push back commit version node failed.", KR(ret), KPC(tx_data_memtable_)); - } - } - - } - return ret; -} - -/** - * 1. This function set an special row for calculating upper trans version - * - */ -int ObTxDataMemtableScanIterator::set_row_with_merged_commit_scns_( - ObCommitSCNsArray &merged_commit_scns, const blocksstable::ObDatumRow *&row) -{ - int ret = OB_SUCCESS; - int64_t pos = 0; - int64_t serialize_size = merged_commit_scns.get_serialize_size(); - - if (OB_FAIL(buf_.reserve(serialize_size))) { - STORAGE_LOG(WARN, "Failed to reserve local buffer", K(ret)); - } else if (OB_FAIL(merged_commit_scns.serialize(buf_.get_ptr(), serialize_size, pos))) { - STORAGE_LOG(WARN, "failed to serialize commit versions", KR(ret), K(serialize_size), K(pos), - K(merged_commit_scns.array_)); - } else { - row_.storage_datums_[TX_DATA_ID_COLUMN].set_int(INT64_MAX); - row_.storage_datums_[TX_DATA_IDX_COLUMN].set_int(0); - row_.storage_datums_[TX_DATA_IDX_COLUMN + 1].set_int(-4096); - row_.storage_datums_[TX_DATA_IDX_COLUMN + 2].set_int(0); - - int64_t total_row_cnt_column = TX_DATA_TOTAL_ROW_CNT_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; - int64_t end_ts_column = TX_DATA_END_TS_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; - int64_t value_column = TX_DATA_VAL_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT; - row_.storage_datums_[total_row_cnt_column].set_int(1); - row_.storage_datums_[end_ts_column].set_int(INT64_MAX); - row_.storage_datums_[value_column].set_string(ObString(serialize_size, buf_.get_ptr())); - - row_.set_first_multi_version_row(); - row_.set_last_multi_version_row(); - row_.set_compacted_multi_version_row(); - row = &row_; - } - return ret; } @@ -700,7 +405,6 @@ int ObTxDataMemtableScanIterator::set_row_with_merged_commit_scns_( int ObTxDataSingleRowGetter::init(const transaction::ObTransID &tx_id) { tx_id_ = tx_id; - return OB_SUCCESS; } @@ -718,6 +422,7 @@ int ObTxDataSingleRowGetter::get_next_row(ObTxData &tx_data) ret = OB_ITER_END; STORAGE_LOG(WARN, "This tablet does not have sstables.", KR(ret), K(table_store)); } else { + tx_data_buffers_.reset(); ret = get_next_row_(sstables, tx_data); } } @@ -730,19 +435,55 @@ int ObTxDataSingleRowGetter::get_next_row_(ObSSTableArray &sstables, ObTxData &t GENERATE_ACCESS_CONTEXT int64_t int_tx_id = tx_id_.get_id(); - // TODO : @gengli A tx data may be divided into multiple rows which means the idx can be greater - // than 0 int64_t idx = 0; GENERATE_ROW_KEY if (OB_SUCC(ret)) { - if (OB_FAIL(get_row_from_sstables_(row_key, sstables, iter_param_, access_context, - tx_data))) { + ObStringHolder temp_buffer; + int64_t total_need_buffer_cnt = 0; + if (OB_FAIL(get_row_from_sstables_(row_key, + sstables, + iter_param_, + access_context, + temp_buffer, + total_need_buffer_cnt))) { if (OB_ITER_END == ret) { STORAGE_LOG(WARN, "tx data not found in sstables", KR(ret), K(tx_id_), K(sstables)); } else { STORAGE_LOG(WARN, "get row from sstables fail.", KR(ret)); } + } else if (OB_FAIL(tx_data_buffers_.reserve(total_need_buffer_cnt))) { + STORAGE_LOG(WARN, "array reserve spaces failed", KR(ret)); + } else if (OB_FAIL(tx_data_buffers_.push_back(std::move(temp_buffer)))) { + STORAGE_LOG(WARN, "push element to reserved array should not fail", KR(ret)); + } else { + int64_t total_need_buffer_cnt2 = 0; + for (int64_t idx = 1; idx < total_need_buffer_cnt && OB_SUCC(ret); ++idx) { + key_datums_[1].set_int(idx); + if (OB_FAIL(row_key.assign(key_datums_, 2))) { + STORAGE_LOG(WARN, "assign row key failed", KR(ret)); + } else if (OB_FAIL(get_row_from_sstables_(row_key, + sstables, + iter_param_, + access_context, + temp_buffer, + total_need_buffer_cnt2))) { + STORAGE_LOG(WARN, "get row from sstable failed", + KR(ret), K(idx), K_(tx_id), K(total_need_buffer_cnt)); + } else if (OB_FAIL(tx_data_buffers_.push_back(std::move(temp_buffer)))) { + STORAGE_LOG(WARN, "push element to reserved array should not fail", KR(ret)); + } else { + if (total_need_buffer_cnt != total_need_buffer_cnt2) { + STORAGE_LOG(ERROR, "multi row's total column count not equal", + KR(ret), K(total_need_buffer_cnt), K(total_need_buffer_cnt2)); + } + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(deserialize_tx_data_from_store_buffers_(tx_data))) { + STORAGE_LOG(WARN, "deserialize from buffers failed", KR(ret), K_(tx_id)); + } + } } } @@ -753,33 +494,39 @@ int ObTxDataSingleRowGetter::get_row_from_sstables_(blocksstable::ObDatumRowkey ObSSTableArray &sstables, const ObTableIterParam &iter_param, ObTableAccessContext &access_context, - ObTxData &tx_data) + ObStringHolder &temp_buffer, + int64_t &total_need_buffer_cnt) { int ret = OB_SUCCESS; ObStoreRowIterator *row_iter = nullptr; ObITable *table = nullptr; - const ObDatumRow *row = nullptr; int tmp_ret = OB_SUCCESS; bool find = false; - + const blocksstable::ObDatumRow *row = nullptr; for (int i = sstables.count() - 1; OB_SUCC(ret) && !find && i >= 0; i--) { if (OB_ISNULL(table = sstables[i])) { ret = OB_ERR_SYS; - STORAGE_LOG(ERROR, "Unexpected null table", K(ret), K(i), K(sstables)); + STORAGE_LOG(ERROR, "Unexpected null table", KR(ret), K(i), K(sstables)); } else if (OB_FAIL(table->get(iter_param, access_context, row_key, row_iter))) { - STORAGE_LOG(WARN, "Failed to get param", K(ret), KPC(table)); + STORAGE_LOG(WARN, "Failed to get param", KR(ret), KPC(table)); } else if (OB_FAIL(row_iter->get_next_row(row))) { if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "Failed to get next row", K(ret), KPC(table)); + STORAGE_LOG(WARN, "Failed to get next row", KR(ret), KPC(table)); } } else if (row->row_flag_.is_not_exist()) { // this tx data not exsit in this sstable, try next one - } else if (FALSE_IT(find = true)) { - } else if (OB_FAIL(deserialize_tx_data_from_store_row_(row, tx_data))) { - STORAGE_LOG(WARN, "deserialize tx data from store row fail.", KR(ret), K(*row)); + } else if (row->storage_datums_[TX_DATA_ID_COLUMN].get_int() != tx_id_) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "read wrong tx data from sstable", + KR(ret), KPC(table), + K(row->storage_datums_[TX_DATA_ID_COLUMN].get_int()), K(tx_id_)); } else { - STORAGE_LOG(DEBUG, "Get tx data from sstable successfully!", K(tx_data)); + find = true; + total_need_buffer_cnt = row->storage_datums_[TX_DATA_TOTAL_ROW_CNT_COLUMN].get_int(); + if (OB_FAIL(temp_buffer.assign(row->storage_datums_[TX_DATA_VAL_COLUMN].get_string()))) { + STORAGE_LOG(WARN, "Failed to copy buffer", KR(ret), KPC(table)); + } } if (OB_NOT_NULL(row_iter)) { @@ -795,21 +542,43 @@ int ObTxDataSingleRowGetter::get_row_from_sstables_(blocksstable::ObDatumRowkey return ret; } -int ObTxDataSingleRowGetter::deserialize_tx_data_from_store_row_(const ObDatumRow *row, - ObTxData &tx_data) +int ObTxDataSingleRowGetter::deserialize_tx_data_from_store_buffers_(ObTxData &tx_data) { int ret = OB_SUCCESS; + int64_t total_buffer_size = 0; int64_t pos = 0; - tx_data.tx_id_ = transaction::ObTransID(row->storage_datums_[TX_DATA_ID_COLUMN].get_int()); - const ObString str = row->storage_datums_[TX_DATA_VAL_COLUMN].get_string(); - if (OB_FAIL(tx_data.deserialize(str.ptr(), str.length(), pos, slice_allocator_))) { - STORAGE_LOG(WARN, "deserialize tx data failed", KR(ret), KPC(row), KPHEX(str.ptr(), str.length())); - hex_dump(str.ptr(), str.length(), true, OB_LOG_LEVEL_WARN); - } else if (!tx_data.is_valid_in_tx_data_table()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "the deserialized tx data is invalid.", KR(ret), K(tx_data)); + char *merge_buffer = nullptr; + for (int64_t idx = 0; idx < tx_data_buffers_.count(); ++idx) { + total_buffer_size += tx_data_buffers_[idx].get_ob_string().length(); + } + if (total_buffer_size <= 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected buffer size", KR(ret), K(total_buffer_size)); + } else if (nullptr == (merge_buffer = (char*)DEFAULT_TX_DATA_ALLOCATOR. + alloc(total_buffer_size))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "fail to alloc merge buffer", KR(ret), K(total_buffer_size)); } else { - // deserialize tx data success + char *p_dest = merge_buffer; + for (int64_t idx = 0; idx < tx_data_buffers_.count(); ++idx) { + OB_ASSERT(p_dest + tx_data_buffers_[idx].get_ob_string().length() <= + merge_buffer + total_buffer_size);// abort or even worse + memcpy(p_dest, tx_data_buffers_[idx].get_ob_string().ptr(), + tx_data_buffers_[idx].get_ob_string().length()); + p_dest += tx_data_buffers_[idx].get_ob_string().length(); + } + tx_data.tx_id_ = tx_id_; + if (OB_FAIL(tx_data.deserialize(merge_buffer, total_buffer_size, pos, slice_allocator_))) { + STORAGE_LOG(WARN, "deserialize tx data failed", + KR(ret), KPHEX(merge_buffer, total_buffer_size)); + hex_dump(merge_buffer, total_buffer_size, true, OB_LOG_LEVEL_WARN); + } else if (!tx_data.is_valid_in_tx_data_table()) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "the deserialized tx data is invalid.", KR(ret), K(tx_data)); + } + } + if (OB_NOT_NULL(merge_buffer)) { + DEFAULT_TX_DATA_ALLOCATOR.free(merge_buffer); } return ret; } @@ -818,7 +587,7 @@ int ObTxDataSingleRowGetter::deserialize_tx_data_from_store_row_(const ObDatumRo /***************************** ObCommitVersionsGetter **********************************/ -int ObCommitVersionsGetter::get_next_row(ObCommitSCNsArray &commit_scns) +int ObCommitVersionsGetter::get_next_row(ObCommitVersionsArray &commit_versions) { int ret = OB_SUCCESS; GENERATE_ACCESS_CONTEXT @@ -838,24 +607,25 @@ int ObCommitVersionsGetter::get_next_row(ObCommitSCNsArray &commit_scns) STORAGE_LOG(WARN, "Failed to get param", K(ret), KPC(table_)); } else if (OB_FAIL(row_iter->get_next_row(row))) { STORAGE_LOG(ERROR, "Failed to get pre-process data for upper trans version calculation", - K(ret), KPC(table_)); + KR(ret), KPC(table_)); } else if (OB_ISNULL(row)) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "unexpected nullptr of row", KR(ret)); } else if (row->row_flag_.is_not_exist()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "Failed to get pre-process data for upper trans version calculation", - K(ret), KPC(table_)); + KR(ret), KPC(table_)); } else { int64_t pos = 0; const ObString str = row->storage_datums_[TX_DATA_VAL_COLUMN].get_string(); - if (OB_FAIL(commit_scns.deserialize(str.ptr(), str.length(), pos))) { - STORAGE_LOG(WARN, "deserialize commit versions array failed.", KR(ret)); - } else if (0 == commit_scns.array_.count()) { + + if (OB_FAIL(commit_versions.deserialize(str.ptr(), str.length(), pos))) { + STORAGE_LOG(WARN, "deserialize commit versions array failed.", KR(ret), KPC(row)); + } else if (0 == commit_versions.array_.count()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "Unexpected empty commit versions array.", KR(ret), KPC(row)); - } else if (!commit_scns.is_valid()) { + } else if (!commit_versions.is_valid()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "invalid cache", KR(ret)); } else { @@ -880,11 +650,12 @@ int ObTxCtxMemtableScanIterator::init(ObTxCtxMemtable *tx_ctx_memtable) // TODO(handora.qc): Optimize the iterator if (OB_ISNULL(ls_tx_ctx_mgr = tx_ctx_memtable->get_ls_tx_ctx_mgr())) { + ret = OB_BAD_NULL_ERROR; STORAGE_LOG(ERROR, "get ls tx ctx mgr failed", KR(ret)); } else if (OB_FAIL(row_.init(allocator_, TX_CTX_TABLE_MAX_COLUMN + SSTABLE_HIDDEN_COLUMN_CNT))) { STORAGE_LOG(WARN, "Failed to init datum row", KR(ret)); } else if (OB_FAIL(buf_.reserve(TX_CTX_BUF_LENGTH))) { - STORAGE_LOG(WARN, "Failed to reserve tx ctx buffer", K(ret)); + STORAGE_LOG(WARN, "Failed to reserve tx ctx buffer", KR(ret)); } else if (OB_FAIL(meta_buf_.reserve(TX_CTX_META_BUF_LENGTH))) { STORAGE_LOG(WARN, "Failed to reserve tx ctx meta buffer", K(ret)); // NB: We must first prepare the rec_scn for ObLSTxCtxMgr and then @@ -892,12 +663,12 @@ int ObTxCtxMemtableScanIterator::init(ObTxCtxMemtable *tx_ctx_memtable) } else if (OB_FAIL(ls_tx_ctx_mgr->refresh_aggre_rec_scn())) { STORAGE_LOG(WARN, "Failed to prepare for dump tx ctx", K(ret)); } else if (OB_FAIL(ls_tx_ctx_iter_.set_ready(ls_tx_ctx_mgr))) { - STORAGE_LOG(WARN, "ls_tx_ctx_iter set_ready failed", K(ret)); + STORAGE_LOG(WARN, "ls_tx_ctx_iter set_ready failed", KR(ret)); } else { row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT); idx_ = 0; is_inited_ = true; - STORAGE_LOG(INFO, "ObTxCtxMemtableScanIterator init succ", K(*this)); + STORAGE_LOG(INFO, "ObTxCtxMemtableScanIterator init succ", KPC(this)); } return ret; @@ -950,7 +721,7 @@ int ObTxCtxMemtableScanIterator::inner_get_next_row(const ObDatumRow *&row) row_buf = buf_.get_ptr() + unmerged_buf_start_pos_; need_merge_length = prev_meta_.get_tx_ctx_serialize_size() - unmerged_buf_start_pos_; if (OB_FAIL(prev_meta_.get_multi_row_next_extent(curr_meta))) { - STORAGE_LOG(WARN, "prev_meta_.get_multi_row_next_extent failed", K(ret), K_(prev_meta)); + STORAGE_LOG(WARN, "prev_meta_.get_multi_row_next_extent failed", KR(ret), K_(prev_meta)); } STORAGE_LOG(DEBUG, "write prev tx ctx unmerged buffer", K(prev_meta_)); } else { @@ -964,11 +735,11 @@ int ObTxCtxMemtableScanIterator::inner_get_next_row(const ObDatumRow *&row) // ceil((double)serialize_size / MAX_VALUE_LENGTH_) (serialize_size + MAX_VALUE_LENGTH_ - 1) / MAX_VALUE_LENGTH_, 0); if (OB_FAIL(buf_.reserve(serialize_size))) { - STORAGE_LOG(WARN, "Failed to reserve local buffer", K(ret)); + STORAGE_LOG(WARN, "Failed to reserve local buffer", KR(ret)); } else { int64_t pos = 0; if (OB_FAIL(ctx_info.serialize(buf_.get_ptr(), serialize_size, pos))) { - STORAGE_LOG(WARN, "failed to serialize ctx_info", K(ret), K(ctx_info), K(pos)); + STORAGE_LOG(WARN, "failed to serialize ctx_info", KR(ret), K(ctx_info), K(pos)); } else { row_buf = buf_.get_ptr(); need_merge_length = serialize_size; @@ -995,11 +766,11 @@ int ObTxCtxMemtableScanIterator::inner_get_next_row(const ObDatumRow *&row) int64_t meta_serialize_size = curr_meta.get_serialize_size(); if (OB_SUCC(ret)) { if (OB_FAIL(meta_buf_.reserve(meta_serialize_size))) { - STORAGE_LOG(WARN, "Failed to reserve tx ctx meta buffer", K(ret)); + STORAGE_LOG(WARN, "Failed to reserve tx ctx meta buffer", KR(ret)); } else { int64_t pos = 0; if (OB_FAIL(curr_meta.serialize(meta_buf_.get_ptr(), meta_serialize_size, pos))) { - STORAGE_LOG(WARN, "Failed to serialize curr_meta", K(ret), K(curr_meta), K(pos)); + STORAGE_LOG(WARN, "Failed to serialize curr_meta", KR(ret), K(curr_meta), K(pos)); } else { // do nothing STORAGE_LOG(DEBUG, "Serialize curr_meta success", K(curr_meta)); diff --git a/src/storage/tx_table/ob_tx_table_iterator.h b/src/storage/tx_table/ob_tx_table_iterator.h index e81a269fb..8b05bfca8 100644 --- a/src/storage/tx_table/ob_tx_table_iterator.h +++ b/src/storage/tx_table/ob_tx_table_iterator.h @@ -13,6 +13,8 @@ #ifndef OCEANBASE_STORAGE_OB_TX_TABLE_ITERATOR #define OCEANBASE_STORAGE_OB_TX_TABLE_ITERATOR +#include "lib/container/ob_array.h" +#include "logservice/archiveservice/ob_archive_util.h" #include "storage/memtable/ob_memtable_iterator.h" #include "storage/tx_table/ob_tx_table_define.h" #include "storage/tx/ob_trans_ctx_mgr.h" @@ -31,8 +33,9 @@ namespace transaction } namespace storage { +class ObTxTable; class ObTxDataMemtable; -class ObTxDataSortListNode; +class ObTxDataLinkNode; class ObSSTableArray; struct TxDataReadSchema; @@ -78,29 +81,25 @@ enum TX_CTX_SSTABLE_COL_IDX : int64_t * */ - class ObTxDataMemtableScanIterator : public memtable::ObIMemtableIterator { private: static const int64_t BUF_LENGTH = 1024; - static int64_t PERIODICAL_SELECT_INTERVAL_NS; public: - ObTxDataMemtableScanIterator(const ObTableIterParam &iter_param) + ObTxDataMemtableScanIterator(const ObTableIterParam &iter_param, const blocksstable::ObDatumRange &range) : is_inited_(false), iter_param_(iter_param), - dump_tx_data_done_(false), - cur_max_commit_version_(), - pre_start_scn_(), - tx_data_row_cnt_(0), + range_(range), + iterate_row_cnt_(0), + start_tx_id_(0), + end_tx_id_(0), + row_cnt_to_dump_(0), pre_tx_data_(nullptr), - arena_allocator_(), cur_node_(nullptr), - row_(), - buf_(arena_allocator_), tx_data_memtable_(nullptr), key_datum_(), - DEBUG_iter_commit_ts_cnt_(0) {} + DEBUG_drop_tx_data_cnt_(0) {} ~ObTxDataMemtableScanIterator() { reset(); } @@ -123,25 +122,27 @@ public: virtual void reuse(); private: - int get_next_tx_data_row_(const blocksstable::ObDatumRow *&row); + int get_next_tx_data_(ObTxData *&tx_data); - int get_next_commit_scn_row_(const blocksstable::ObDatumRow *&row); + int drop_and_get_tx_data_(ObTxData *&tx_data); - int prepare_commit_scn_list_(); + int get_next_commit_version_row_(const blocksstable::ObDatumRow *&row); - int periodical_get_next_commit_scn_(ObCommitSCNsArray::Node &node); + int prepare_commit_version_list_(); - int fill_in_cur_commit_scns_(ObCommitSCNsArray &cur_commit_scns); + int periodical_get_next_commit_version_(ObCommitVersionsArray::Node &node); - int get_past_commit_scns_(ObCommitSCNsArray &past_commit_scns); + int fill_in_cur_commit_versions_(ObCommitVersionsArray &cur_commit_versions); - int deserialize_commit_scns_array_from_row_(const blocksstable::ObDatumRow *row, ObCommitSCNsArray &past_commit_scns); + int get_past_commit_versions_(ObCommitVersionsArray &past_commit_versions); + int deserialize_commit_versions_array_from_row_(const blocksstable::ObDatumRow *row, + ObCommitVersionsArray &past_commit_versions); int merge_cur_and_past_commit_verisons_(const share::SCN recycle_scn, - ObCommitSCNsArray &cur_commit_scns, - ObCommitSCNsArray &past_commit_scns, - ObCommitSCNsArray &merged_commit_scns); + ObCommitVersionsArray &cur_commit_versions, + ObCommitVersionsArray &past_commit_versions, + ObCommitVersionsArray &merged_commit_versions); /** * @brief get node from data_arr and push_back it to merged_arr @@ -156,41 +157,59 @@ private: int merge_pre_process_node_(const int64_t step_len, const share::SCN start_scn_limit, const share::SCN recycle_scn, - const ObIArray &data_arr, + const ObIArray &data_arr, share::SCN &max_commit_version, - ObIArray &merged_arr); + ObIArray &merged_arr); - int set_row_with_merged_commit_scns_(ObCommitSCNsArray &merged_commit_scns, + int set_row_with_merged_commit_versions_(ObCommitVersionsArray &merged_commit_versions, const blocksstable::ObDatumRow *&row); - int DEBUG_check_past_and_cur_arr(ObCommitSCNsArray &cur_commit_versions, - ObCommitSCNsArray &past_commit_versions); + int init_iterate_range_(ObTxDataMemtable *tx_data_memtable); - int DEBUG_try_calc_upper_and_check_(ObCommitSCNsArray &merged_commit_versions); + int DEBUG_check_past_and_cur_arr(ObCommitVersionsArray &cur_commit_versions, + ObCommitVersionsArray &past_commit_versions); + + int DEBUG_try_calc_upper_and_check_(ObCommitVersionsArray &merged_commit_versions); int DEBUG_fake_calc_upper_trans_version(const share::SCN sstable_end_scn, share::SCN &upper_trans_version, - ObCommitSCNsArray &merged_commit_versions); + ObCommitVersionsArray &merged_commit_versions); void DEBUG_print_start_scn_list_(); - void DEBUG_print_merged_commit_versions_(ObCommitSCNsArray &merged_commit_versions); + + int init_iterate_count_(ObTxDataMemtable *tx_data_memtable); private: + class TxData2DatumRowConverter { + public: + TxData2DatumRowConverter() : + serialize_buffer_(nullptr), buffer_len_(0), tx_data_(nullptr), generate_size_(0) {} + ~TxData2DatumRowConverter() { reset(); } + OB_NOINLINE int init(ObTxData *tx_data); + void reset(); + int generate_next_now(const blocksstable::ObDatumRow *&row); + TO_STRING_KV(KP_(serialize_buffer), K_(buffer_len), KPC_(tx_data), + K_(generate_size), K_(datum_row)); + private: + char *serialize_buffer_; + int64_t buffer_len_; + ObTxData *tx_data_; + int64_t generate_size_; + blocksstable::ObDatumRow datum_row_; + }; bool is_inited_; const ObTableIterParam &iter_param_; - bool dump_tx_data_done_; - share::SCN cur_max_commit_version_; - share::SCN pre_start_scn_; - int64_t tx_data_row_cnt_; + const blocksstable::ObDatumRange &range_; + int64_t iterate_row_cnt_; + int64_t start_tx_id_; + int64_t end_tx_id_; + int64_t row_cnt_to_dump_; ObTxData *pre_tx_data_; - ObArenaAllocator arena_allocator_; - ObTxDataSortListNode *cur_node_; - blocksstable::ObDatumRow row_; - ObTxLocalBuffer buf_; + ObTxDataLinkNode *cur_node_; + TxData2DatumRowConverter tx_data_2_datum_converter_; ObTxDataMemtable *tx_data_memtable_; blocksstable::ObStorageDatum key_datum_; - int64_t DEBUG_iter_commit_ts_cnt_; - share::SCN DEBUG_last_start_scn_; + int64_t DEBUG_drop_tx_data_cnt_; }; /** @@ -219,15 +238,17 @@ private: ObSSTableArray &sstables, const ObTableIterParam &iter_param, ObTableAccessContext &access_context, - ObTxData &tx_data); - int deserialize_tx_data_from_store_row_(const blocksstable::ObDatumRow *row, ObTxData &tx_data); + ObStringHolder &temp_buffer, + int64_t &total_need_buffer_cnt); + OB_NOINLINE int deserialize_tx_data_from_store_buffers_(ObTxData &tx_data); private: - const ObTableIterParam & iter_param_; + const ObTableIterParam &iter_param_; SliceAllocator &slice_allocator_; transaction::ObTransID tx_id_; ObArenaAllocator arena_allocator_; blocksstable::ObStorageDatum key_datums_[2]; + ObArray tx_data_buffers_; }; /** @@ -239,7 +260,7 @@ public: : iter_param_(iter_param), table_(table), key_datums_() {} virtual ~ObCommitVersionsGetter() {} - int get_next_row(ObCommitSCNsArray &commit_scns); + int get_next_row(ObCommitVersionsArray &commit_versions); private: const ObTableIterParam &iter_param_; @@ -302,7 +323,6 @@ private: transaction::ObLSTxCtxIterator ls_tx_ctx_iter_; const int64_t MAX_VALUE_LENGTH_; bool is_inited_; - }; } // namespace storage 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 3501e4ced..adb1b85b6 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 @@ -265,6 +265,7 @@ _max_schema_slot_num _migrate_block_verify_level _minor_compaction_amplification_factor _minor_compaction_interval +_mvcc_gc_using_min_txn_snapshot _ob_ddl_timeout _ob_elr_fast_freeze_threshold _ob_enable_fast_freeze @@ -304,6 +305,7 @@ _sqlexec_disable_hash_based_distagg_tiv _storage_meta_memory_limit_percentage _temporary_file_io_area_size _trace_control_info +_tx_result_retention _upgrade_stage _xa_gc_interval _xa_gc_timeout diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result index 1dbef3630..32d11143f 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result @@ -217,6 +217,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 413 __all_spatial_reference_systems 0 201001 1 416 __all_column_checksum_error_info 0 201001 1 443 __all_tenant_rewrite_rules 0 201001 1 +444 __all_reserved_snapshot 0 201001 1 445 __all_cluster_event_history 0 201001 1 10001 __tenant_virtual_all_table 2 201001 1 10002 __tenant_virtual_table_column 2 201001 1 diff --git a/unittest/storage/memtable/test_memtable_basic.cpp b/unittest/storage/memtable/test_memtable_basic.cpp index 531332ee3..402513a79 100644 --- a/unittest/storage/memtable/test_memtable_basic.cpp +++ b/unittest/storage/memtable/test_memtable_basic.cpp @@ -95,6 +95,22 @@ int ObMvccRow::check_double_insert_(const share::SCN , { return OB_SUCCESS; } +} + +namespace concurrent_control +{ +int check_sequence_set_violation(const concurrent_control::ObWriteFlag , + const int64_t , + const ObTransID , + const blocksstable::ObDmlFlag , + const int64_t , + const ObTransID , + const blocksstable::ObDmlFlag , + const int64_t ) +{ + return OB_SUCCESS; +} + } // end memtable class TestMemtable : public testing::Test @@ -210,6 +226,7 @@ public: ObStoreCtx store_ctx; ObTxSnapshot snapshot; ObTxTableGuard tx_table_guard; + concurrent_control::ObWriteFlag write_flag; tx_table_guard.init((ObTxTable*)0x100); snapshot.version_.convert_for_gts(snapshot_version); store_ctx.mvcc_acc_ctx_.init_write(trans_ctx_, @@ -220,7 +237,8 @@ public: tx_table_guard, snapshot, INT64_MAX, - INT64_MAX); + INT64_MAX, + write_flag); ObTableStoreIterator table_iter; store_ctx.table_iter_ = &table_iter; ObStoreRow write_row; diff --git a/unittest/storage/test_compaction_policy.cpp b/unittest/storage/test_compaction_policy.cpp index 32d1b3d2c..cc2541a42 100644 --- a/unittest/storage/test_compaction_policy.cpp +++ b/unittest/storage/test_compaction_policy.cpp @@ -626,7 +626,11 @@ int TestCompactionPolicy::prepare_freeze_info( if (OB_ISNULL(mgr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mgr is unexpected null", K(ret)); - } else if (OB_FAIL(mgr->update_info(snapshot_gc_ts, freeze_infos, snapshots, min_major_snapshot, changed))) { + } else if (OB_FAIL(mgr->update_info(snapshot_gc_ts, + freeze_infos, + snapshots, + min_major_snapshot, + changed))) { LOG_WARN("failed to update info", K(ret)); } return ret; diff --git a/unittest/storage/tx/it/test_tx.cpp b/unittest/storage/tx/it/test_tx.cpp index 9564d42b8..4346793d4 100644 --- a/unittest/storage/tx/it/test_tx.cpp +++ b/unittest/storage/tx/it/test_tx.cpp @@ -27,6 +27,21 @@ using namespace ::testing; using namespace transaction; using namespace share; +namespace concurrent_control +{ +int check_sequence_set_violation(const concurrent_control::ObWriteFlag , + const int64_t , + const ObTransID , + const blocksstable::ObDmlFlag , + const int64_t , + const ObTransID , + const blocksstable::ObDmlFlag , + const int64_t ) +{ + return OB_SUCCESS; +} +} + namespace common { void* ObGMemstoreAllocator::alloc(AllocHandle& handle, int64_t size) diff --git a/unittest/storage/tx/it/tx_node.cpp b/unittest/storage/tx/it/tx_node.cpp index 672a8bd23..2ac42fb78 100644 --- a/unittest/storage/tx/it/tx_node.cpp +++ b/unittest/storage/tx/it/tx_node.cpp @@ -484,8 +484,10 @@ int ObTxNode::write(ObTxDesc &tx, write_store_ctx.ls_ = &mock_ls_; write_store_ctx.ls_id_ = ls_id_; write_store_ctx.table_iter_ = iter; + concurrent_control::ObWriteFlag write_flag; OZ(txs_.get_write_store_ctx(tx, snapshot, + write_flag, write_store_ctx)); write_store_ctx.mvcc_acc_ctx_.tx_table_guard_.init(&fake_tx_table_); ObArenaAllocator allocator; @@ -517,8 +519,10 @@ int ObTxNode::write_begin(ObTxDesc &tx, iter->add_tables(&mtb, 1); write_store_ctx.ls_id_ = ls_id_; write_store_ctx.table_iter_ = iter; + concurrent_control::ObWriteFlag write_flag; OZ(txs_.get_write_store_ctx(tx, snapshot, + write_flag, write_store_ctx)); return ret; } diff --git a/unittest/storage/tx/mock_utils/basic_fake_define.h b/unittest/storage/tx/mock_utils/basic_fake_define.h index 826339f82..3e97f5fc5 100644 --- a/unittest/storage/tx/mock_utils/basic_fake_define.h +++ b/unittest/storage/tx/mock_utils/basic_fake_define.h @@ -13,6 +13,9 @@ #ifndef OCEANBASE_TRANSACTION_TEST_BASIC_FAKE_DEFINE_ #define OCEANBASE_TRANSACTION_TEST_BASIC_FAKE_DEFINE_ +#define protected public +#define private public + #include "storage/tx/ob_trans_define.h" #include "storage/tx_table/ob_tx_table.h" #include "lib/utility/ob_defer.h" @@ -27,74 +30,92 @@ namespace oceanbase { using namespace share; using namespace memtable; + + namespace transaction { class ObFakeTxDataTable : public ObTxDataTable { public: - ObFakeTxDataTable() : map_() { IGNORE_RETURN map_.init(); } + ObFakeTxDataTable() : map_(1 << 20 /*2097152*/) + { + IGNORE_RETURN map_.init(); + ObMemAttr mem_attr; + mem_attr.label_ = "TX_DATA_TABLE"; + mem_attr.tenant_id_ = 1; + mem_attr.ctx_id_ = ObCtxIds::DEFAULT_CTX_ID; + ObMemtableMgrHandle memtable_mgr_handle; + OB_ASSERT(OB_SUCCESS == slice_allocator_.init( + sizeof(ObTxData), OB_MALLOC_NORMAL_BLOCK_SIZE, common::default_blk_alloc, mem_attr)); + slice_allocator_.set_nway(ObTxDataTable::TX_DATA_MAX_CONCURRENCY); + is_inited_ = true; + } virtual int init(ObLS *ls, ObTxCtxTable *tx_ctx_table) override - { return OB_SUCCESS; } + { + return OB_SUCCESS; + } virtual int start() override { return OB_SUCCESS; } virtual void stop() override {} virtual void reset() override {} virtual void destroy() override {} - virtual int alloc_tx_data(ObTxData *&tx_data) override + virtual int alloc_tx_data(ObTxDataGuard &tx_data_guard) override { - return map_.alloc_value(tx_data); + void *ptr = slice_allocator_.alloc(); + ObTxData *tx_data = new (ptr) ObTxData(); + tx_data->ref_cnt_ = 100; + tx_data->slice_allocator_ = &slice_allocator_; + tx_data->flag_ = 269381; + tx_data_guard.init(tx_data); + return OB_ISNULL(tx_data) ? OB_ALLOCATE_MEMORY_FAILED : OB_SUCCESS; } - virtual int deep_copy_tx_data(ObTxData *from, ObTxData *&to) override + virtual int deep_copy_tx_data(const ObTxDataGuard &from_guard, ObTxDataGuard &to_guard) override { int ret = OB_SUCCESS; - OZ (map_.alloc_value(to)); + void *ptr = slice_allocator_.alloc(); + ObTxData *to = new (ptr) ObTxData(); + ObTxData *from = (ObTxData*)from_guard.tx_data(); + to->ref_cnt_ = 100; + to->slice_allocator_ = &slice_allocator_; + to->flag_ = 269381; + to_guard.init(to); OX (*to = *from); OZ (deep_copy_undo_status_list_(from->undo_status_list_, to->undo_status_list_)); return ret; } - virtual void free_tx_data(ObTxData *tx_data) override + virtual void free_tx_data(ObTxData *tx_data) { - map_.free_value(tx_data); } virtual int alloc_undo_status_node(ObUndoStatusNode *&undo_status_node) override { - undo_status_node = new ObUndoStatusNode(); + void *ptr = ob_malloc(TX_DATA_SLICE_SIZE); + undo_status_node = new (ptr) ObUndoStatusNode(); return OB_SUCCESS; } virtual int free_undo_status_node(ObUndoStatusNode *&undo_status_node) override { - delete undo_status_node; return OB_SUCCESS; } virtual int insert(ObTxData *&tx_data) override { int ret = OB_SUCCESS; - ObTxData *old = NULL; - if (OB_SUCC(map_.get(tx_data->tx_id_, old))) { - OX (map_.revert(old)); - OZ (map_.del(tx_data->tx_id_)); - } else if (OB_ENTRY_NOT_EXIST == ret) { - ret = OB_SUCCESS; - } - OZ (map_.insert_and_get(tx_data->tx_id_, tx_data)); - OX (map_.revert(tx_data)); + OZ (map_.insert(tx_data->tx_id_, tx_data)); return ret; } virtual int check_with_tx_data(const ObTransID tx_id, ObITxDataCheckFunctor &fn) override { int ret = OB_SUCCESS; - ObTxData *tx_data = NULL; - OZ (map_.get(tx_id, tx_data)); - OZ (fn(*tx_data)); - if (OB_NOT_NULL(tx_data)) { map_.revert(tx_data); } + ObTxDataGuard tx_data_guard; + OZ (map_.get(tx_id, tx_data_guard)); + OZ (fn(*tx_data_guard.tx_data())); if (OB_ENTRY_NOT_EXIST == ret) { ret = OB_TRANS_CTX_NOT_EXIST; } return ret; } - common::ObLinkHashMap map_; + ObTxDataHashMap map_; }; class ObFakeTxTable : public ObTxTable { public: ObFakeTxTable() : ObTxTable(tx_data_table_) {} -private: +public: ObFakeTxDataTable tx_data_table_; }; diff --git a/unittest/storage/tx/ob_mock_tx_ctx.cpp b/unittest/storage/tx/ob_mock_tx_ctx.cpp index f6026ef2c..5d2b545bb 100644 --- a/unittest/storage/tx/ob_mock_tx_ctx.cpp +++ b/unittest/storage/tx/ob_mock_tx_ctx.cpp @@ -430,4 +430,5 @@ void MockObTxCtx::set_exiting_() } } // end namespace transaction + } // end namespace oceanbase diff --git a/unittest/storage/tx/test_simple_tx_ctx.cpp b/unittest/storage/tx/test_simple_tx_ctx.cpp index 007251724..dbcb4ac2c 100644 --- a/unittest/storage/tx/test_simple_tx_ctx.cpp +++ b/unittest/storage/tx/test_simple_tx_ctx.cpp @@ -14,6 +14,8 @@ #include #define private public #define protected public +#define UNITTEST + #include "storage/tx/ob_mock_tx_ctx.h" namespace oceanbase diff --git a/unittest/storage/tx_table/test_tx_ctx_table.cpp b/unittest/storage/tx_table/test_tx_ctx_table.cpp index 13e56ce29..de7e8c339 100644 --- a/unittest/storage/tx_table/test_tx_ctx_table.cpp +++ b/unittest/storage/tx_table/test_tx_ctx_table.cpp @@ -14,6 +14,7 @@ #define protected public #define private public +#define UNITTEST #include #include "storage/meta_mem/ob_tenant_meta_mem_mgr.h" @@ -259,7 +260,7 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr) ObSliceAlloc slice_allocator; ObMemAttr attr; attr.tenant_id_ = MTL_ID(); - slice_allocator.init(TX_DATA_SLICE_SIZE, OB_MALLOC_NORMAL_BLOCK_SIZE, common::default_blk_alloc, attr); + slice_allocator.init(sizeof(ObTxData), OB_MALLOC_NORMAL_BLOCK_SIZE, common::default_blk_alloc, attr); ObTxPalfParam palf_param((logservice::ObLogHandler *)(0x01));