[FEAT MERGE] Big Txn Execution Optimization
This commit is contained in:
parent
4dfd67d8fb
commit
561432c689
@ -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
|
||||
}
|
||||
|
3
deps/oblib/src/lib/oblog/ob_log_module.h
vendored
3
deps/oblib/src/lib/oblog/ob_log_module.h
vendored
@ -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, \
|
||||
|
@ -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)) {
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <gtest/gtest.h>
|
||||
#define protected public
|
||||
#define private public
|
||||
#define UNITTEST
|
||||
|
||||
#include "mtlenv/mock_tenant_module_env.h"
|
||||
#include "storage/tablelock/ob_lock_memtable.h"
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <gtest/gtest.h>
|
||||
#define protected public
|
||||
#define private public
|
||||
#define UNITTEST
|
||||
|
||||
#include "mtlenv/mock_tenant_module_env.h"
|
||||
#include "storage/tablelock/ob_obj_lock.h"
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <gtest/gtest.h>
|
||||
#define protected public
|
||||
#define private public
|
||||
#define UNITTEST
|
||||
|
||||
#include "mtlenv/mock_tenant_module_env.h"
|
||||
#include "storage/tablelock/ob_obj_lock.h"
|
||||
|
@ -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)
|
||||
|
@ -14,6 +14,7 @@
|
||||
|
||||
#define protected public
|
||||
#define private public
|
||||
#define UNITTEST
|
||||
|
||||
#include <iostream>
|
||||
#include <thread>
|
||||
@ -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<common::ObStoreRange, range_cnt> 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);
|
||||
|
@ -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)
|
||||
|
@ -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<observer::ObSimpleServer>(env_prefix_);
|
||||
cluster_ = std::make_shared<observer::ObSimpleServer>(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));
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
||||
|
9
mittest/simple_server/env/ob_simple_server.h
vendored
9
mittest/simple_server/env/ob_simple_server.h
vendored
@ -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_;
|
||||
|
@ -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_);
|
||||
|
||||
|
651
mittest/simple_server/fast_commit_report.cpp
Normal file
651
mittest/simple_server/fast_commit_report.cpp
Normal file
@ -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 <gtest/gtest.h>
|
||||
#include <thread>
|
||||
#include <iostream>
|
||||
#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<const void *>(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<memtable::ObMemtable *>(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();
|
||||
}
|
320
mittest/simple_server/rewrite_function_for_test_big_tx_data.cpp
Normal file
320
mittest/simple_server/rewrite_function_for_test_big_tx_data.cpp
Normal file
@ -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 <iostream>
|
||||
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<ObStoreRange> 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;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
215
mittest/simple_server/test_big_tx_data.cpp
Normal file
215
mittest/simple_server/test_big_tx_data.cpp
Normal file
@ -0,0 +1,215 @@
|
||||
#include <chrono>
|
||||
#include <gtest/gtest.h>
|
||||
#include <thread>
|
||||
#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 <iostream>
|
||||
|
||||
#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();
|
||||
}
|
751
mittest/simple_server/test_mvcc_gc.cpp
Normal file
751
mittest/simple_server/test_mvcc_gc.cpp
Normal file
@ -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 <gtest/gtest.h>
|
||||
#include <thread>
|
||||
#include <iostream>
|
||||
#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<uint64_t> &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<concurrency_control::ObMultiVersionSnapshotInfo> &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<concurrency_control::ObMultiVersionSnapshotInfo> 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();
|
||||
}
|
@ -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());
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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_;
|
||||
|
@ -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);
|
||||
|
@ -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<uint64_t, COMMON_COLUMN_NUM> 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<uint64_t> &full_column_ids = replace_iter.get_column_ids();
|
||||
const int64_t rowkey_column_cnt = table_operation.entity().get_rowkey_size();
|
||||
ObSEArray<uint64_t, COMMON_COLUMN_NUM> 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<uint64_t> &column_ids = replace_iter.get_column_ids();
|
||||
ObSEArray<uint64_t, COMMON_COLUMN_NUM> 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();
|
||||
|
@ -154,10 +154,7 @@ int ObRootMinorFreeze::get_tenant_server_list(uint64_t tenant_id,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObRootMinorFreeze::try_minor_freeze(const ObIArray<uint64_t> &tenant_ids,
|
||||
const ObIArray<ObAddr> &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<uint64_t> &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<ObLSReplicaLocation> &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<uint64_t> &tenant_id
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObAddr, 256> 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<uint64_t> &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<ObAddr> &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<ObAddr> &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<ObAddr> &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));
|
||||
|
@ -46,11 +46,7 @@ public:
|
||||
void start();
|
||||
void stop();
|
||||
int destroy();
|
||||
|
||||
int try_minor_freeze(const common::ObIArray<uint64_t> &tenant_ids,
|
||||
const common::ObIArray<common::ObAddr> &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<MinorFreezeParam> &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<uint64_t> &tenant_ids,
|
||||
const common::ObZone &zone,
|
||||
const common::ObIArray<common::ObAddr> &server_list,
|
||||
@ -95,10 +95,6 @@ private:
|
||||
int init_params_by_server(const common::ObIArray<common::ObAddr> &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;
|
||||
|
@ -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));
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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) {
|
||||
|
@ -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";
|
||||
|
@ -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',
|
||||
|
@ -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:
|
||||
|
@ -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<int(const ObServerStatus &status)> &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<int(const ObServerStatus &status)> &functor)
|
||||
{
|
||||
return trace_map_.for_each_server_status(functor);
|
||||
}
|
||||
|
@ -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<int(const ObServerStatus &status)> &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<int(const ObServerStatus &status)> &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;
|
||||
|
@ -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,)\
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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<uint64_t> 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<uint64_t> tenant_ids_;
|
||||
common::ObSArray<common::ObAddr> server_list_;
|
||||
common::ObZone zone_;
|
||||
common::ObTabletID tablet_id_;
|
||||
share::ObLSID ls_id_;
|
||||
};
|
||||
|
||||
struct ObSyncPGPartitionMTFinishArg
|
||||
|
@ -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));
|
||||
|
@ -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* \
|
||||
)
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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++) {
|
||||
|
@ -441,7 +441,7 @@ END_P SET_VAR DELIMITER
|
||||
%type <node> create_view_stmt view_name opt_column_list opt_table_id opt_tablet_id view_select_stmt opt_check_option
|
||||
%type <node> name_list
|
||||
%type <node> 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 <node> ls opt_tenant_list_and_tablet_id ls_server_or_server_or_zone_or_tenant add_or_alter_zone_option
|
||||
%type <node> ls opt_tenant_list_or_ls_or_tablet_id ls_server_or_server_or_zone_or_tenant add_or_alter_zone_option
|
||||
%type <node> opt_tenant_list_v2
|
||||
%type <node> suspend_or_resume tenant_name opt_tenant_name cache_name opt_cache_name file_id opt_file_id cancel_task_type
|
||||
%type <node> 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
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
static int get_tenant_ids(const ParseNode &t_node, common::ObIArray<uint64_t> &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<uint64_t> &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);
|
||||
|
@ -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<uint64_t> &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<uint64_t> 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
|
||||
|
@ -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;
|
||||
|
@ -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;}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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(
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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 {
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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",
|
||||
|
@ -116,6 +116,7 @@ public:
|
||||
COMPACTION_POLICY,
|
||||
GET_TABLE_SCHEMA,
|
||||
CALC_PROGRESSIVE_PARAM,
|
||||
PRE_PROCESS_TX_TABLE,
|
||||
GET_PARALLEL_RANGE,
|
||||
EXECUTE,
|
||||
CREATE_SSTABLE,
|
||||
|
@ -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);
|
||||
|
@ -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,
|
||||
|
0
src/storage/compaction/ob_tenant_tablet_scheduler.cpp
Normal file → Executable file
0
src/storage/compaction/ob_tenant_tablet_scheduler.cpp
Normal file → Executable file
20
src/storage/compaction/ob_tx_table_merge_task.cpp
Normal file → Executable file
20
src/storage/compaction/ob_tx_table_merge_task.cpp
Normal file → Executable file
@ -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<storage::ObITable *> &tables = ctx.tables_handle_.get_tables();
|
||||
for (int i = 0; OB_SUCC(ret) && i < tables.count(); i++) {
|
||||
if (OB_FAIL(static_cast<ObTxDataMemtable *>(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;
|
||||
|
1
src/storage/compaction/ob_tx_table_merge_task.h
Normal file → Executable file
1
src/storage/compaction/ob_tx_table_merge_task.h
Normal file → Executable file
@ -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);
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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<ObMultiVersionGCStatus>(static_cast<uint64_t>(a) | static_cast<uint64_t>(b));
|
||||
}
|
||||
|
||||
inline ObMultiVersionGCStatus operator & (ObMultiVersionGCStatus a, ObMultiVersionGCStatus b)
|
||||
{
|
||||
return static_cast<ObMultiVersionGCStatus>(static_cast<uint64_t>(a) & static_cast<uint64_t>(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<int(const share::SCN snapshot_version,
|
||||
const ObMultiVersionSnapshotType snapshot_type,
|
||||
const ObMultiVersionGCStatus status,
|
||||
const int64_t create_time,
|
||||
const ObAddr addr)>;
|
||||
|
||||
// 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<ObMultiVersionSnapshotInfo> &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<ObMultiVersionSnapshotInfo> &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<ObAddr> &snapshot_servers);
|
||||
int reclaim_(const ObArray<ObAddr> &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
|
@ -437,6 +437,20 @@ int ObFreezer::tablet_freeze(const ObTabletID &tablet_id, ObFuture<int> *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<int> *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<int> *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<int> *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;
|
||||
|
@ -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<int> *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
|
||||
|
@ -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,
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -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++) {
|
||||
|
@ -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
|
||||
|
@ -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_;
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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_;
|
||||
|
@ -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 &&
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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 =====================
|
||||
|
@ -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<transaction::ObPartTransCtx *>(get_trans_ctx());
|
||||
part_ctx->clear_block_frozen_memtable();
|
||||
|
||||
ret = remove();
|
||||
return ret;
|
||||
}
|
||||
|
@ -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<keybtree::BtreeIterator> iter;
|
||||
ObStoreRowkeyWrapper scan_start_key_wrapper(&ObStoreRowkey::MIN_STORE_ROWKEY);
|
||||
ObStoreRowkeyWrapper scan_end_key_wrapper(&ObStoreRowkey::MAX_STORE_ROWKEY);
|
||||
iter.reset();
|
||||
const_cast<ObMemtableKey *>(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_);
|
||||
|
@ -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);
|
||||
|
121
src/storage/memtable/ob_concurrent_control.cpp
Normal file
121
src/storage/memtable/ob_concurrent_control.cpp
Normal file
@ -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
|
115
src/storage/memtable/ob_concurrent_control.h
Normal file
115
src/storage/memtable/ob_concurrent_control.h
Normal file
@ -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
|
@ -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<ObITable *> *stores = nullptr;
|
||||
common::ObSEArray<ObITable *, 4> 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));
|
||||
}
|
||||
|
@ -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_();
|
||||
|
||||
|
@ -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<transaction::ObPartTransCtx *>(mem_ctx_->get_trans_ctx());
|
||||
part_ctx->set_block_frozen_memtable(static_cast<memtable::ObMemtable *>(iter->get_memtable()));
|
||||
}
|
||||
} else {
|
||||
bool fake_fill = false;
|
||||
if (MutatorType::MUTATOR_ROW == iter->get_mutator_type()) {
|
||||
|
@ -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;
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user