[FEAT MERGE] transaction pdml support optimization

Co-authored-by: yyy-hust <yyy.hust@gmail.com>
Co-authored-by: SanmuWangZJU <sanmuwang.ws@gmail.com>
This commit is contained in:
chinaxing
2023-12-16 12:42:52 +00:00
committed by ant-ob-hengtang
parent eaa353f503
commit 438a70b2b8
169 changed files with 8960 additions and 3921 deletions

View File

@ -42,7 +42,7 @@ class ObTxLogBlockBuilder
{
public:
ObTxLogBlockBuilder(const int64_t tx_id, const uint64_t cluster_id)
: tx_id_(tx_id), cluster_id_(cluster_id), log_entry_no_(0), tx_log_block_() {}
: tx_id_(tx_id), cluster_id_(cluster_id), cluster_version_(DATA_VERSION_4_3_0_0), log_entry_no_(0), tx_log_block_() {}
~ObTxLogBlockBuilder() {}
public:
int next_log_block();
@ -55,7 +55,8 @@ public:
int64_t get_log_entry_no() { return log_entry_no_; }
private:
TxID tx_id_;
uint64 cluster_id_;
uint64_t cluster_id_;
int64_t cluster_version_;
int64_t log_entry_no_;
ObTxLogBlock tx_log_block_;
};
@ -211,10 +212,10 @@ private:
int ObTxLogBlockBuilder::next_log_block()
{
int ret = OB_SUCCESS;
ObTxLogBlockHeader block_header(cluster_id_, log_entry_no_, tx_id_, common::ObAddr());
tx_log_block_.reset();
if (OB_FAIL(tx_log_block_.init(tx_id_, block_header))) {
ObTxLogBlockHeader &block_header = tx_log_block_.get_header();
block_header.init(cluster_id_, cluster_version_, log_entry_no_, tx_id_, common::ObAddr());
if (OB_FAIL(tx_log_block_.init_for_fill())) {
LOG_ERROR("init tx_log_block_ failed", KR(ret), K_(tx_id), K(block_header));
} else {
log_entry_no_ ++;
@ -321,10 +322,11 @@ void ObTxLogGenerator::gen_commit_log()
EXPECT_EQ(OB_SUCCESS, ls_info_arr.push_back(ls_info1));
EXPECT_EQ(OB_SUCCESS, ls_info_arr.push_back(ls_info2));
}
ObArray<uint8_t> checksum_signature;
ObTxCommitLog commit_log(
commit_version,
checksum,
checksum_signature,
inc_ls_arr,
mds_arr,
trans_type_,

File diff suppressed because it is too large Load Diff

View File

@ -469,6 +469,11 @@ public:
scn.set_max();
return OB_SUCCESS;
}
int get_max_decided_scn_as_leader(share::SCN &scn) const
{
scn.set_max();
return OB_SUCCESS;
}
int get_max_decided_log_ts_ns(int64_t &log_ts)
{
log_ts = INT64_MAX;

View File

@ -367,10 +367,10 @@ int TestCompactionPolicy::mock_memtable(
snapshot_scn.convert_for_tx(snapshot_version);
memtable->snapshot_version_ = snapshot_scn;
memtable->write_ref_cnt_ = 0;
memtable->unsynced_cnt_ = 0;
memtable->unsubmitted_cnt_ = 0;
memtable->is_tablet_freeze_ = true;
memtable->state_ = ObMemtableState::MINOR_FROZEN;
memtable->set_resolve_active_memtable_left_boundary(true);
memtable->set_resolved_active_memtable_left_boundary(true);
memtable->set_frozen();
memtable->location_ = storage::checkpoint::ObFreezeCheckpointLocation::PREPARE;
}

View File

@ -40,6 +40,7 @@ storage_unittest(test_ob_tx_log)
storage_unittest(test_ob_timestamp_service)
storage_unittest(test_ob_trans_rpc)
storage_unittest(test_ob_tx_msg)
storage_unittest(test_undo_action)
storage_unittest(test_ob_id_meta)
storage_unittest(test_ob_standby_read)
storage_unittest(test_ob_standby_read_transfer)
@ -47,3 +48,7 @@ storage_unittest(test_ob_trans_tlog)
add_subdirectory(it)
storage_unittest(test_dup_table_lease)
storage_unittest(test_redo_submitter)
storage_unittest(test_trans_callback_mgr_fill_redo)
storage_unittest(test_misc)

View File

@ -76,7 +76,6 @@ int check_sequence_set_violation(const concurrent_control::ObWriteFlag ,
return OB_SUCCESS;
}
}
class ObTestTx : public ::testing::Test
{
public:
@ -84,8 +83,8 @@ public:
{
oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION);
ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001);
const uint64_t tv = ObTimeUtility::current_time();
ObCurTraceId::set(&tv);
ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1",2023);
ObCurTraceId::init(ip_port);
GCONF._ob_trans_rpc_timeout = 500;
ObClockGenerator::init();
const testing::TestInfo* const test_info =
@ -2414,6 +2413,51 @@ TEST_F(ObTestTx, interrupt_get_read_snapshot)
ROLLBACK_TX(n1, tx);
}
TEST_F(ObTestTx, rollback_with_branch_savepoint)
{
START_ONE_TX_NODE(n1);
PREPARE_TX(n1, tx);
PREPARE_TX_PARAM(tx_param);
CREATE_IMPLICIT_SAVEPOINT(n1, tx, tx_param, global_sp1);
CREATE_BRANCH_SAVEPOINT(n1, tx, 100, sp_b100_1);
ASSERT_EQ(OB_SUCCESS, n1->write(tx, 100, 111, 100));
CREATE_BRANCH_SAVEPOINT(n1, tx, 200, sp_b200_1);
ASSERT_EQ(OB_SUCCESS, n1->write(tx, 200, 211, 200));
ASSERT_EQ(OB_SUCCESS, n1->write(tx, 101, 112, 100));
ASSERT_EQ(OB_SUCCESS, n1->write(tx, 500, 505)); // global write
ASSERT_EQ(OB_SUCCESS, n1->write(tx, 201, 212, 200));
// rollback branch 200
ASSERT_EQ(OB_SUCCESS, ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, sp_b200_1, 2000*1000));
// check branch 100 is readable
int64_t val = 0;
ASSERT_EQ(OB_SUCCESS, n1->read(tx, 101, val));
ASSERT_EQ(val, 112);
// check global write is readable
ASSERT_EQ(OB_SUCCESS, n1->read(tx, 500, val));
ASSERT_EQ(val, 505);
// check branch 200 is un-readable
ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 200, val));
ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 201, val));
// write with branch 200
ASSERT_EQ(OB_SUCCESS, n1->write(tx, 206, 602, 200));
// rollback branch 100
ASSERT_EQ(OB_SUCCESS, ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, sp_b100_1, 2000*1000));
// check global write is readable
ASSERT_EQ(OB_SUCCESS, n1->read(tx, 500, val));
ASSERT_EQ(val, 505);
// check branch 200 is readable
ASSERT_EQ(OB_SUCCESS, n1->read(tx, 206, val));
ASSERT_EQ(val, 602);
// check branch 100 is un-readable
ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 100, val));
ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 101, val));
// rollback global
ASSERT_EQ(OB_SUCCESS, ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, global_sp1, 2000 * 1000));
// check global and branch 200 is un-readable
ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 500, val));
ASSERT_EQ(OB_ENTRY_NOT_EXIST, n1->read(tx, 206, val));
ROLLBACK_TX(n1, tx);
}
////
/// APPEND NEW TEST HERE, USE PRE DEFINED MACRO IN FILE `test_tx.dsl`
/// SEE EXAMPLE: TEST_F(ObTestTx, rollback_savepoint_timeout)
@ -2423,6 +2467,10 @@ TEST_F(ObTestTx, interrupt_get_read_snapshot)
int main(int argc, char **argv)
{
uint64_t checksum = 1100101;
uint64_t c = 0;
uint64_t checksum1 = ob_crc64(checksum, (void*)&c, sizeof(uint64_t));
uint64_t checksum2 = ob_crc64(c, (void*)&checksum, sizeof(uint64_t));
int64_t tx_id = 21533427;
uint64_t h = murmurhash(&tx_id, sizeof(tx_id), 0);
system("rm -rf test_tx.log*");
@ -2433,6 +2481,6 @@ int main(int argc, char **argv)
"test_tx.log"); // audit
logger.set_log_level(OB_LOG_LEVEL_DEBUG);
::testing::InitGoogleTest(&argc, argv);
TRANS_LOG(INFO, "mmhash:", K(h));
TRANS_LOG(INFO, "mmhash:", K(h), K(checksum1), K(checksum2));
return RUN_ALL_TESTS();
}

View File

@ -130,7 +130,7 @@ TEST_F(ObTestTxCtx, DelayAbort)
ASSERT_EQ(OB_SUCCESS, ls_tx_ctx_mgr->get_tx_ctx(tx.tx_id_, false /*for_replay*/, tx_ctx));
GCONF._private_buffer_size = 1;
// ASSERT_EQ(OB_SUCCESS, tx_ctx->submit_log_impl_(ObTxLogType::TX_REDO_LOG));
ASSERT_EQ(OB_SUCCESS, tx_ctx->submit_redo_log(false /*is_freeze*/));
ASSERT_EQ(OB_SUCCESS, tx_ctx->submit_redo_after_write(false, ObTxSEQ()));
TRANS_LOG(INFO, "[TEST] after submit redo", K(tx_ctx->trans_id_),
K(tx_ctx->exec_info_.max_applied_log_ts_));
n1->wait_all_redolog_applied();

View File

@ -53,8 +53,12 @@
ObTxSEQ sp; \
ASSERT_EQ(OB_SUCCESS, n1->create_implicit_savepoint(tx, tx_param, sp));
#define CREATE_BRANCH_SAVEPOINT(n1, tx, branch, sp) \
ObTxSEQ sp; \
ASSERT_EQ(OB_SUCCESS, n1->create_branch_savepoint(tx, branch, sp));
#define ROLLBACK_TO_IMPLICIT_SAVEPOINT(n1, tx, sp, timeout_us) \
n1->rollback_to_implicit_savepoint(tx, sp, n1->ts_after_us(timeout_us), nullptr);
n1->rollback_to_implicit_savepoint(tx, sp, n1->ts_after_us(timeout_us), nullptr)
#define INJECT_LINK_FAILURE(n1, n2) \
ASSERT_EQ(OB_SUCCESS, bus_.inject_link_failure(n1->addr_, n2->addr_)); \

View File

@ -648,8 +648,8 @@ public:
{
oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION);
ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001);
const uint64_t tv = ObTimeUtility::current_time();
ObCurTraceId::set(&tv);
ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1", 2023);
ObCurTraceId::init(ip_port);
GCONF._ob_trans_rpc_timeout = 500;
ObClockGenerator::init();
omt::the_ctrl_of_enable_transaction_free_route = true;

View File

@ -22,6 +22,13 @@
} while(0);
namespace oceanbase {
namespace common {
int ObClusterVersion::get_tenant_data_version(const uint64_t tenant_id, uint64_t &data_version)
{
data_version = DATA_CURRENT_VERSION;
return OB_SUCCESS;
}
}
namespace share
{
void* ObMemstoreAllocator::alloc(AllocHandle& handle, int64_t size, const int64_t expire_ts)
@ -152,6 +159,7 @@ int ObTxNode::start() {
fake_tx_log_adapter_ = new ObFakeTxLogAdapter();
OZ(fake_tx_log_adapter_->start());
}
get_ts_mgr_().reset();
OZ(msg_consumer_.start());
OZ(txs_.start());
OZ(create_ls_(ls_id_));
@ -243,6 +251,12 @@ ObTxNode::~ObTxNode() __attribute__((optnone)) {
ObTenantEnv::set_tenant(&tenant_);
OZ(txs_.tx_ctx_mgr_.revert_ls_tx_ctx_mgr(fake_tx_table_.tx_ctx_table_.ls_tx_ctx_mgr_));
fake_tx_table_.tx_ctx_table_.ls_tx_ctx_mgr_ = nullptr;
bool is_tx_clean = false;
int retry_cnt = 0;
do {
usleep(2000);
txs_.block_tx(ls_id_, is_tx_clean);
} while(!is_tx_clean && ++retry_cnt < 1000);
OX(txs_.stop());
OZ(txs_.wait_());
if (role_ == Leader && fake_tx_log_adapter_) {
@ -561,7 +575,7 @@ int ObTxNode::atomic_write(ObTxDesc &tx, const int64_t key, const int64_t value,
}
return ret;
}
int ObTxNode::write(ObTxDesc &tx, const int64_t key, const int64_t value)
int ObTxNode::write(ObTxDesc &tx, const int64_t key, const int64_t value, const int16_t branch)
{
int ret = OB_SUCCESS;
ObTxReadSnapshot snapshot;
@ -569,13 +583,14 @@ int ObTxNode::write(ObTxDesc &tx, const int64_t key, const int64_t value)
tx.isolation_,
ts_after_ms(50),
snapshot));
OZ(write(tx, snapshot, key, value));
OZ(write(tx, snapshot, key, value, branch));
return ret;
}
int ObTxNode::write(ObTxDesc &tx,
const ObTxReadSnapshot &snapshot,
const int64_t key,
const int64_t value)
const int64_t value,
const int16_t branch)
{
TRANS_LOG(INFO, "write", K(key), K(value), K(snapshot), K(tx), KPC(this));
int ret = OB_SUCCESS;
@ -589,6 +604,7 @@ int ObTxNode::write(ObTxDesc &tx,
write_store_ctx.ls_ = &mock_ls_;
write_store_ctx.ls_id_ = ls_id_;
write_store_ctx.table_iter_ = iter;
write_store_ctx.branch_ = branch;
concurrent_control::ObWriteFlag write_flag;
OZ(txs_.get_write_store_ctx(tx,
snapshot,
@ -709,20 +725,28 @@ int ObTxNode::replay(const void *buffer,
const int64_t ts_ns)
{
ObTenantEnv::set_tenant(&tenant_);
int ret = OB_SUCCESS;
logservice::ObLogBaseHeader base_header;
int64_t tmp_pos = 0;
const char *log_buf = static_cast<const char *>(buffer);
if (OB_FAIL(base_header.deserialize(log_buf, nbytes, tmp_pos))) {
LOG_WARN("log base header deserialize error", K(ret));
} else if (OB_FAIL(ObFakeTxReplayExecutor::execute(&mock_ls_, mock_ls_.get_tx_svr(), log_buf, nbytes,
tmp_pos, lsn, ts_ns, base_header.get_replay_hint(),
ls_id_, tenant_id_, memtable_))) {
LOG_WARN("replay tx log error", K(ret), K(lsn), K(ts_ns));
} else {
LOG_INFO("replay tx log succ", K(ret), K(lsn), K(ts_ns));
share::SCN log_scn;
log_scn.convert_for_tx(ts_ns);
ObFakeTxReplayExecutor executor(&mock_ls_,
ls_id_,
tenant_id_,
mock_ls_.get_tx_svr(),
lsn,
log_scn,
base_header);
executor.set_memtable(memtable_);
if (OB_FAIL(executor.execute(log_buf, nbytes, tmp_pos))) {
LOG_WARN("replay tx log error", K(ret), K(lsn), K(ts_ns));
} else {
LOG_INFO("replay tx log succ", K(ret), K(lsn), K(ts_ns));
}
}
return ret;
}

View File

@ -138,11 +138,12 @@ public:
int read(const ObTxReadSnapshot &snapshot,
const int64_t key,
int64_t &value);
int write(ObTxDesc &tx, const int64_t key, const int64_t value);
int write(ObTxDesc &tx, const int64_t key, const int64_t value, const int16_t branch = 0);
int write(ObTxDesc &tx,
const ObTxReadSnapshot &snapshot,
const int64_t key,
const int64_t value);
const int64_t value,
const int16_t branch = 0);
int atomic_write(ObTxDesc &tx, const int64_t key, const int64_t value,
const int64_t expire_ts, const ObTxParam &tx_param);
int replay(const void *buffer, const int64_t nbytes, const palf::LSN &lsn, const int64_t ts_ns);
@ -167,6 +168,7 @@ public:
DELEGATE_TENANT_WITH_RET(txs_, abort_tx, int);
DELEGATE_TENANT_WITH_RET(txs_, submit_commit_tx, int);
DELEGATE_TENANT_WITH_RET(txs_, get_read_snapshot, int);
DELEGATE_TENANT_WITH_RET(txs_, create_branch_savepoint, int);
DELEGATE_TENANT_WITH_RET(txs_, create_implicit_savepoint, int);
DELEGATE_TENANT_WITH_RET(txs_, create_explicit_savepoint, int);
DELEGATE_TENANT_WITH_RET(txs_, rollback_to_explicit_savepoint, int);

View File

@ -242,6 +242,11 @@ public:
get_gts_error_ = OB_SUCCESS;
}
public:
void reset() {
get_gts_error_ = OB_SUCCESS;
elapse_waiting_mode_ = false;
get_gts_waiting_mode_ = false;
}
int update_gts(const uint64_t tenant_id, const int64_t gts, bool &update) { return OB_SUCCESS; }
int get_gts(const uint64_t tenant_id,
const MonotonicTs stc,
@ -262,7 +267,7 @@ public:
ret = OB_EAGAIN;
}
}
TRANS_LOG(INFO, "get gts end", K(ret), K(gts_), K(gts), K(&gts_));
TRANS_LOG(INFO, "get gts end", K(ret), K(gts_), K(gts), K(get_gts_waiting_mode_));
return ret;
}
@ -402,6 +407,7 @@ public:
ObSpScLinkQueue apply_task_queue_arr[TASK_QUEUE_CNT];
ObSpScLinkQueue replay_task_queue_arr[TASK_QUEUE_CNT];
share::SCN max_submit_scn_ = share::SCN::invalid_scn();
share::SCN max_committed_scn_ = share::SCN::invalid_scn();
void run1() {
while(true) {
@ -413,6 +419,7 @@ public:
ObLink *task = apply_task_queue_arr[i].pop();
if (task) {
++process_cnt;
max_committed_scn_ = static_cast<ApplyCbTask*>(task)->cb_->__get_scn();
static_cast<ApplyCbTask*>(task)->cb_->on_success();
delete task;
ATOMIC_DEC(&inflight_cnt_);
@ -558,7 +565,10 @@ public:
}
return OB_SUCCESS;
}
int get_palf_committed_max_scn(share::SCN &scn) const {
scn = max_committed_scn_;
return OB_SUCCESS;
}
int get_append_mode_initial_scn(share::SCN &ref_scn) {
int ret = OB_SUCCESS;
ref_scn = share::SCN::invalid_scn();
@ -603,54 +613,26 @@ class ObFakeTxReplayExecutor : public ObTxReplayExecutor
{
public:
ObFakeTxReplayExecutor(storage::ObLS *ls,
const share::ObLSID &ls_id,
const uint64_t tenant_id,
storage::ObLSTxService *ls_tx_srv,
const palf::LSN &lsn,
const share::SCN &log_timestamp)
: ObTxReplayExecutor(ls, ls_tx_srv, lsn, log_timestamp) {memtable_ = nullptr;}
~ObFakeTxReplayExecutor() {}
static int execute(storage::ObLS *ls,
storage::ObLSTxService *ls_tx_srv,
const char *buf,
const int64_t size,
const int skip_pos,
const palf::LSN &lsn,
const int64_t &log_timestamp,
const int64_t &replay_hint,
const share::ObLSID &ls_id,
const int64_t &tenant_id,
memtable::ObMemtable* memtable)
{
int ret = OB_SUCCESS;
share::SCN log_scn;
log_scn.convert_for_gts(log_timestamp);
ObFakeTxReplayExecutor replay_executor(ls, ls_tx_srv, lsn, log_scn);
if (OB_ISNULL(ls) || OB_ISNULL(ls_tx_srv) || OB_ISNULL(buf) || size <= 0
|| 0 >= log_timestamp || INT64_MAX == log_timestamp || !lsn.is_valid()) {
ret = OB_INVALID_ARGUMENT;
TRANS_LOG(ERROR, "invaild arguments", K(replay_executor), K(buf), K(size));
} else if (replay_executor.set_memtable(memtable)) {
} else if (OB_FAIL(replay_executor.do_replay_(buf,
size,
skip_pos,
replay_hint,
ls_id,
tenant_id))) {
TRANS_LOG(ERROR, "replay_executor.do_replay failed",
K(replay_executor), K(buf), K(size), K(skip_pos), K(replay_hint), K(ls_id), K(tenant_id));
hex_dump(buf, size, true, OB_LOG_LEVEL_INFO);
}
return ret;
}
private:
const share::SCN &log_timestamp,
const logservice::ObLogBaseHeader &base_header)
: ObTxReplayExecutor(ls, ls_id, tenant_id, ls_tx_srv, lsn, log_timestamp, base_header)
{ memtable_ = nullptr; }
~ObFakeTxReplayExecutor() { }
int set_memtable(memtable::ObMemtable* memtable)
{
memtable_ = memtable;
return OB_SUCCESS;
}
int execute(const char *buf,
const int64_t size,
const int skip_pos)
{
return do_replay_(buf, size, skip_pos);
}
int replay_one_row_in_memtable_(memtable::ObMutatorRowHeader& row_head,
memtable::ObMemtableMutatorIterator *mmi_ptr) override
{
@ -659,13 +641,12 @@ private:
storeCtx.ls_id_ = ctx_->get_ls_id();
storeCtx.mvcc_acc_ctx_.tx_ctx_ = ctx_;
storeCtx.mvcc_acc_ctx_.mem_ctx_ = mt_ctx_;
storeCtx.replay_log_scn_ = log_ts_ns_;
storeCtx.tablet_id_ = row_head.tablet_id_;
storeCtx.mvcc_acc_ctx_.tx_id_ = ctx_->get_trans_id();
switch (row_head.mutator_type_) {
case memtable::MutatorType::MUTATOR_ROW: {
if (OB_FAIL(memtable_->replay_row(storeCtx, mmi_ptr_))) {
if (OB_FAIL(memtable_->replay_row(storeCtx, log_ts_ns_, mmi_ptr_))) {
TRANS_LOG(WARN, "[Replay Tx] replay row error", K(ret));
} else {
TRANS_LOG(INFO, "[Replay Tx] replay row in memtable success");

View File

@ -75,6 +75,11 @@ public:
UNUSED(scn);
return OB_SUCCESS;
}
int get_palf_committed_max_scn(share::SCN &scn) const
{
UNUSED(scn);
return OB_SUCCESS;
}
int get_append_mode_initial_scn(share::SCN &ref_scn) {
int ret = OB_SUCCESS;
ref_scn = share::SCN::invalid_scn();

View File

@ -85,15 +85,12 @@ TEST_F(TestLSLogWriter, submit_start_working_log)
ObTxStartWorkingLog sw_log(tmp_ref);
int64_t test_leader_epoch = 1308;
ObTxLogBlockHeader block_header;
ASSERT_EQ(OB_SUCCESS, ls_log_writer.init(tmp_tenant_id, TEST_LS_ID, &tx_log_adapter,
(ObLSTxCtxMgr *)&tmp_mgr));
ASSERT_EQ(OB_SUCCESS, ls_log_writer.submit_start_working_log(test_leader_epoch, log_ts));
ASSERT_EQ(true, tx_log_adapter.get_log(log_ts.get_val_for_gts(), log_string));
ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(log_string.c_str(), log_string.size(),
replay_hint, block_header));
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(log_string.c_str(), log_string.size()));
ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header));
EXPECT_EQ(ObTxLogType::TX_START_WORKING_LOG, log_header.get_tx_log_type());
ASSERT_EQ(OB_SUCCESS, replay_block.deserialize_log_body(sw_log));

View File

@ -0,0 +1,131 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define protected public
#include <gtest/gtest.h>
#include "share/ob_errno.h"
#include "lib/oblog/ob_log.h"
#include "storage/memtable/ob_memtable_context.h"
namespace oceanbase
{
using namespace common;
using namespace transaction;
namespace unittest
{
class TestObTxMisc : public ::testing::Test
{
public :
virtual void SetUp() {}
virtual void TearDown() {}
};
TEST_F(TestObTxMisc, multiple_checksum_collapse_for_commit_log)
{
TRANS_LOG(INFO, "called", "func", test_info_->name());
// only one checksum
{
uint64_t checksum0 = 12323221;
ObArrayHelper<uint64_t> arr(1, &checksum0, 1);
uint8_t signature0 = 0;
ObArrayHelper<uint8_t> sig(1, &signature0, 1);
uint64_t result = 0;
memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig);
EXPECT_EQ(result, 12323221);
EXPECT_EQ(0, sig.count());
}
// multiple, but only one is valid
{
uint64_t checksum[64] = {12323221};
for (int i = 1; i < 64; i++) checksum[i] = 1;
ObArrayHelper<uint64_t> arr(64, checksum, 64);
uint8_t signature[64];
ObArrayHelper<uint8_t> sig(64, signature, 0);
uint64_t result = 0;
memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig);
EXPECT_EQ(result, 12323221);
EXPECT_EQ(0, sig.count());
// valid is in middle
checksum[0] = 1;
checksum[13] = 34443;
memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig);
EXPECT_EQ(result, 34443);
EXPECT_EQ(0, sig.count());
}
// multiple, multiple valid: 1, 13
{
uint64_t checksum[64] = {12323221};
for (int i = 1; i < 64; i++) checksum[i] = 1;
checksum[13] = 34443;
ObArrayHelper<uint64_t> arr(64, checksum, 64);
uint8_t signature[64];
ObArrayHelper<uint8_t> sig(64, signature, 0);
uint64_t result = 0;
memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig);
EXPECT_GT(result, 1);
EXPECT_NE(result, 12323221);
EXPECT_NE(result, 34443);
EXPECT_EQ(64, sig.count());
EXPECT_EQ(12323221 & 0xFF, sig.at(0));
EXPECT_EQ(34443 & 0xFF, sig.at(13));
}
// multiple, multiple valid, 18,21
{
uint64_t checksum[64];
for (int i = 0; i < 64; i++) checksum[i] = 1;
checksum[18] = 34443;
checksum[21] = 34444;
ObArrayHelper<uint64_t> arr(64, checksum, 64);
uint8_t signature[64];
ObArrayHelper<uint8_t> sig(64, signature, 0);
uint64_t result = 0;
memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig);
EXPECT_GT(result, 1);
EXPECT_NE(result, 34443);
EXPECT_NE(result, 34444);
EXPECT_EQ(64, sig.count());
EXPECT_EQ(34443 & 0xFF, sig.at(18));
EXPECT_EQ(34444 & 0xFF, sig.at(21));
}
// multiple, all is valid
{
uint64_t checksum[64] = {12323221};
for (int i = 1; i < 64; i++) checksum[i] = 1 + i;
ObArrayHelper<uint64_t> arr(64, checksum, 64);
uint8_t signature[64];
ObArrayHelper<uint8_t> sig(64, signature, 0);
uint64_t result = 0;
memtable::ObMemtableCtx::convert_checksum_for_commit_log(arr, result, sig);
EXPECT_NE(result, 12323221);
EXPECT_GT(result, 1);
EXPECT_EQ(64, sig.count());
EXPECT_EQ(12323221 & 0xFF, sig.at(0));
}
}
}//end of unittest
}//end of oceanbase
using namespace oceanbase;
using namespace oceanbase::common;
int main(int argc, char **argv)
{
int ret = 1;
ObLogger &logger = ObLogger::get_logger();
logger.set_file_name("test_ob_tx_misc.log", true);
logger.set_log_level(OB_LOG_LEVEL_INFO);
testing::InitGoogleTest(&argc, argv);
ret = RUN_ALL_TESTS();
return ret;
}

View File

@ -15,7 +15,8 @@
#define private public
#include "storage/tx/ob_tx_log.h"
#include "logservice/ob_log_base_header.h"
#include "lib/container/ob_array_helper.h"
void ob_abort (void) __THROW {}
namespace oceanbase
{
using namespace common;
@ -35,8 +36,7 @@ public:
//const TEST
TxID TEST_TX_ID = 1024;
int64_t TEST_CLUSTER_VERSION = 1;
int64_t TEST_LOG_NO = 1;
int64_t TEST_CLUSTER_VERSION = DATA_VERSION_4_3_0_0;
ObAddr TEST_ADDR(ObAddr::VER::IPV4,"1.0.0.1",606);
int TEST_TRANS_TYPE = 1;
int TEST_SESSION_ID = 56831;
@ -54,10 +54,12 @@ common::ObString TEST_TRCE_INFO("trace_info_test");
LogOffSet TEST_LOG_OFFSET(10);
int64_t TEST_COMMIT_VERSION = 190878;
int64_t TEST_CHECKSUM = 29890209;
ObArray<uint8_t> TEST_CHECKSUM_SIGNATURE_ARRAY;
int64_t TEST_SCHEMA_VERSION = 372837;
int64_t TEST_TX_EXPIRED_TIME = 12099087;
int64_t TEST_LOG_ENTRY_NO = 1233;
auto TEST_MAX_SUBMITTED_SEQ_NO = ObTxSEQ(12345, 0);
ObTxSEQ TEST_MAX_SUBMITTED_SEQ_NO = ObTxSEQ(12345, 0);
ObTxSEQ TEST_SERIAL_FINAL_SEQ_NO = ObTxSEQ(12346, 0);
LSKey TEST_LS_KEY;
ObXATransID TEST_XID;
@ -108,39 +110,54 @@ OB_TX_SERIALIZE_MEMBER(NewTestLog, compat_bytes_, tx_id_1, tx_id_2, tx_id_3);
TEST_F(TestObTxLog, tx_log_block_header)
{
TRANS_LOG(INFO, "called", "func", test_info_->name());
TxID id = 0;
int64_t pos = 0;
ObTxLogBlock fill_block, replay_block;
ObTxLogBlockHeader fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header));
ObTxLogBlockHeader &fill_block_header = fill_block.get_header();
fill_block_header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
fill_block_header.set_serial_final();
ASSERT_TRUE(fill_block_header.is_serial_final());
ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill());
fill_block.seal(TEST_TX_ID);
// check log_block_header
char *buf = fill_block.get_buf();
logservice::ObLogBaseHeader base_header_1;
logservice::ObLogBaseHeader base_header_2;
pos = 0;
base_header_1.deserialize(buf, base_header_1.get_serialize_size(), pos);
EXPECT_EQ(base_header_1.get_log_type() , ObTxLogBlock::DEFAULT_LOG_BLOCK_TYPE);
EXPECT_EQ(base_header_1.get_replay_hint(), TEST_TX_ID);
ObTxLogBlockHeader replay_block_header;
ASSERT_EQ(OB_SUCCESS,
replay_block.init_with_header(buf,
fill_block.get_size(),
id,
replay_block_header));
ObTxLogBlockHeader &replay_block_header = replay_block.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(buf, fill_block.get_size()));
uint64_t tmp_cluster_id = replay_block_header.get_org_cluster_id();
EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id);
EXPECT_EQ(id, TEST_TX_ID);
EXPECT_EQ(replay_block.get_log_base_header().get_replay_hint(), TEST_TX_ID);
EXPECT_EQ(TEST_CLUSTER_VERSION, replay_block_header.get_cluster_version());
EXPECT_EQ(TEST_LOG_ENTRY_NO, replay_block_header.get_log_entry_no());
EXPECT_EQ(fill_block_header.flags(), replay_block_header.flags());
EXPECT_TRUE(replay_block_header.is_serial_final());
fill_block.reuse(id, replay_block_header);
// reuse
fill_block.get_header().init(TEST_ORG_CLUSTER_ID + 1, TEST_CLUSTER_VERSION + 1, TEST_LOG_ENTRY_NO + 1, ObTransID(TEST_TX_ID + 1), TEST_ADDR);
fill_block.reuse_for_fill();
fill_block.seal(TEST_TX_ID + 1);
buf = fill_block.get_buf();
pos = 0;
logservice::ObLogBaseHeader base_header_2;
base_header_2.deserialize(buf, base_header_2.get_serialize_size(), pos);
EXPECT_EQ(base_header_2.get_log_type() , ObTxLogBlock::DEFAULT_LOG_BLOCK_TYPE);
EXPECT_EQ(base_header_2.get_replay_hint(), TEST_TX_ID);
EXPECT_EQ(base_header_2.get_replay_hint(), TEST_TX_ID + 1);
ObTxLogBlock replay_block2;
ObTxLogBlockHeader &replay_block_header2 = replay_block2.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block2.init_for_replay(buf, fill_block.get_size(), pos));
EXPECT_EQ(TEST_ORG_CLUSTER_ID + 1, replay_block_header2.get_org_cluster_id());
EXPECT_EQ(TEST_CLUSTER_VERSION + 1, replay_block_header2.get_cluster_version());
EXPECT_EQ(TEST_LOG_ENTRY_NO + 1, replay_block_header2.get_log_entry_no());
EXPECT_EQ(fill_block_header.flags(), replay_block_header2.flags());
EXPECT_FALSE(replay_block_header2.is_serial_final());
}
TEST_F(TestObTxLog, tx_log_body_except_redo)
@ -196,10 +213,12 @@ TEST_F(TestObTxLog, tx_log_body_except_redo)
TEST_LAST_SCN,
TEST_MAX_SUBMITTED_SEQ_NO,
TEST_CLUSTER_VERSION,
TEST_XID);
TEST_XID,
TEST_SERIAL_FINAL_SEQ_NO);
ObTxPrepareLog filll_prepare(TEST_LS_ARRAY, TEST_LOG_OFFSET);
ObTxCommitLog fill_commit(share::SCN::base_scn(),
TEST_CHECKSUM,
TEST_CHECKSUM_SIGNATURE_ARRAY,
TEST_LS_ARRAY,
TEST_TX_BUFFER_NODE_ARRAY,
TEST_TRANS_TYPE,
@ -209,8 +228,9 @@ TEST_F(TestObTxLog, tx_log_body_except_redo)
ObTxAbortLog fill_abort(TEST_TX_BUFFER_NODE_ARRAY);
ObTxRecordLog fill_record(TEST_LOG_OFFSET, TEST_LOG_OFFSET_ARRY);
ObTxLogBlockHeader header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, header));
ObTxLogBlockHeader &header = fill_block.get_header();
header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill());
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_active_state));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit_state));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(filll_prepare));
@ -218,13 +238,11 @@ TEST_F(TestObTxLog, tx_log_body_except_redo)
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_clear));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_abort));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_record));
TxID id = 0;
fill_block.seal(TEST_TX_ID);
ObTxLogHeader tx_log_header;
ObTxLogBlockHeader block_header;
ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(fill_block.get_buf(), fill_block.get_size(), id, block_header));
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size()));
uint64_t tmp_cluster_id = block_header.get_org_cluster_id();
uint64_t tmp_cluster_id = replay_block.get_header().get_org_cluster_id();
EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id);
ObTxActiveInfoLogTempRef active_temp_ref;
@ -247,7 +265,9 @@ TEST_F(TestObTxLog, tx_log_body_except_redo)
EXPECT_EQ(TEST_LAST_OP_SN, replay_active_state.get_last_op_sn());
EXPECT_EQ(TEST_FIRST_SCN, replay_active_state.get_first_seq_no());
EXPECT_EQ(TEST_LAST_SCN, replay_active_state.get_last_seq_no());
EXPECT_EQ(TEST_CLUSTER_VERSION, replay_active_state.get_cluster_version());
EXPECT_EQ(0, replay_active_state.get_cluster_version());
EXPECT_EQ(TEST_XID, replay_active_state.get_xid());
EXPECT_EQ(TEST_SERIAL_FINAL_SEQ_NO, replay_active_state.get_serial_final_seq_no());
ObTxCommitInfoLogTempRef commit_state_temp_ref;
ObTxCommitInfoLog replay_commit_state(commit_state_temp_ref);
@ -322,28 +342,29 @@ TEST_F(TestObTxLog, tx_log_body_redo)
TEST_EPOCH);
ObTxCommitLog fill_commit(share::SCN::base_scn(),
TEST_CHECKSUM,
TEST_CHECKSUM_SIGNATURE_ARRAY,
TEST_LS_ARRAY,
TEST_TX_BUFFER_NODE_ARRAY,
TEST_TRANS_TYPE,
TEST_LOG_OFFSET,
TEST_INFO_ARRAY);
ObTxLogBlockHeader fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header));
ObTxLogBlockHeader &fill_block_header = fill_block.get_header();
fill_block_header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill());
ObString TEST_MUTATOR_BUF("FFF");
int64_t mutator_pos = 0;
ObTxRedoLog fill_redo(TEST_LOG_NO, TEST_CLUSTER_VERSION);
ObTxRedoLog fill_redo(TEST_CLUSTER_VERSION);
ASSERT_EQ(OB_SUCCESS, fill_block.prepare_mutator_buf(fill_redo));
ASSERT_EQ(OB_SUCCESS,
serialization::encode(fill_redo.get_mutator_buf(),
fill_redo.get_mutator_size(),
mutator_pos,
TEST_MUTATOR_BUF));
ASSERT_EQ(OB_SUCCESS, serialization::encode(fill_redo.get_mutator_buf(),
fill_redo.get_mutator_size(),
mutator_pos,
TEST_MUTATOR_BUF));
ASSERT_EQ(OB_SUCCESS, fill_block.finish_mutator_buf(fill_redo, mutator_pos));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit_state));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit));
fill_block.seal(TEST_TX_ID);
mutator_pos = 0;
TxID id = 0;
ObTxLogHeader log_header;
@ -351,11 +372,12 @@ TEST_F(TestObTxLog, tx_log_body_redo)
ObTxRedoLogTempRef redo_temp_ref;
ObTxRedoLog replay_redo(redo_temp_ref);
ObTxLogBlockHeader replay_block_header;
ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(fill_block.get_buf(), fill_block.get_size(), id, replay_block_header));
ObTxLogBlockHeader &replay_block_header = replay_block.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size()));
uint64_t tmp_cluster_id = replay_block_header.get_org_cluster_id();
EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id);
EXPECT_EQ(TEST_CLUSTER_VERSION, replay_block_header.get_cluster_version());
ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header));
EXPECT_EQ(ObTxLogType::TX_REDO_LOG, log_header.get_tx_log_type());
@ -366,15 +388,13 @@ TEST_F(TestObTxLog, tx_log_body_redo)
K(fill_redo.get_mutator_buf()),
K(replay_redo.get_replay_mutator_buf()),
K(replay_redo.get_mutator_size()));
ASSERT_EQ(OB_SUCCESS,
serialization::decode(replay_redo.get_replay_mutator_buf(),
replay_redo.get_mutator_size(),
mutator_pos,
replay_mutator_buf));
ASSERT_EQ(OB_SUCCESS, serialization::decode(replay_redo.get_replay_mutator_buf(),
replay_redo.get_mutator_size(),
mutator_pos,
replay_mutator_buf));
EXPECT_EQ(TEST_MUTATOR_BUF, replay_mutator_buf);
// EXPECT_EQ(TEST_CLOG_ENCRYPT_INFO,replay_redo.get_clog_encrypt_info());
// EXPECT_EQ(TEST_LOG_NO,replay_redo.get_log_no());
EXPECT_EQ(TEST_CLUSTER_VERSION,replay_redo.get_cluster_version());
EXPECT_EQ(replay_redo.get_cluster_version(), 0);
ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header));
EXPECT_EQ(ObTxLogType::TX_COMMIT_INFO_LOG, log_header.get_tx_log_type());
ASSERT_EQ(OB_SUCCESS, replay_block.get_next_log(log_header));
@ -382,8 +402,8 @@ TEST_F(TestObTxLog, tx_log_body_redo)
//ignore replay log, only need commit log
ObTxLogBlockHeader replay_block_header_2;
ASSERT_EQ(OB_SUCCESS, replay_block_2.init_with_header(fill_block.get_buf(), fill_block.get_size(), id, replay_block_header_2));
ObTxLogBlockHeader &replay_block_header_2 = replay_block_2.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block_2.init_for_replay(fill_block.get_buf(), fill_block.get_size()));
tmp_cluster_id = replay_block_header_2.get_org_cluster_id();
EXPECT_EQ(TEST_ORG_CLUSTER_ID, tmp_cluster_id);
@ -398,7 +418,6 @@ TEST_F(TestObTxLog, tx_log_body_redo)
// replay_mutator_buf));
// EXPECT_EQ(TEST_MUTATOR_BUF, replay_mutator_buf);
// EXPECT_EQ(TEST_CLOG_ENCRYPT_INFO,replay_redo.get_clog_encrypt_info());
// EXPECT_EQ(TEST_LOG_NO,replay_redo.get_log_no());
// EXPECT_EQ(TEST_CLUSTER_VERSION,replay_redo.get_cluster_version());
ASSERT_EQ(OB_SUCCESS, replay_block_2.get_next_log(log_header));
EXPECT_EQ(ObTxLogType::TX_COMMIT_INFO_LOG, log_header.get_tx_log_type());
@ -492,8 +511,9 @@ TEST_F(TestObTxLog, test_default_log_deserialize)
// ObTxLogBlockHeader fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO,
// ObTransID(TEST_TX_ID));
ObTxLogBlockHeader fill_block_header;
ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header));
ObTxLogBlockHeader &fill_block_header = fill_block.get_header();
fill_block_header.init(1, TEST_CLUSTER_VERSION, 2, ObTransID(3), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill());
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_active_state));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_commit_state));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_prepare));
@ -501,14 +521,14 @@ TEST_F(TestObTxLog, test_default_log_deserialize)
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_clear));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_abort));
ASSERT_EQ(OB_SUCCESS, fill_block.add_new_log(fill_record));
fill_block.seal(TEST_TX_ID);
TxID id = 0;
int64_t fill_member_cnt = 0;
int64_t replay_member_cnt = 0;
ObTxLogHeader tx_log_header;
ObTxLogBlockHeader replay_block_header;
ASSERT_EQ(OB_SUCCESS, replay_block.init_with_header(fill_block.get_buf(), fill_block.get_size(),
id, replay_block_header));
ObTxLogBlockHeader &replay_block_header = replay_block.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size()));
fill_member_cnt = fill_block_header.compat_bytes_.total_obj_cnt_;
EXPECT_EQ(fill_block_header.get_org_cluster_id(), replay_block_header.get_org_cluster_id());
replay_member_cnt++;
@ -518,7 +538,11 @@ TEST_F(TestObTxLog, test_default_log_deserialize)
replay_member_cnt++;
EXPECT_EQ(fill_block_header.get_scheduler(), replay_block_header.get_scheduler());
replay_member_cnt++;
EXPECT_EQ(replay_member_cnt, fill_member_cnt);
EXPECT_EQ(fill_block_header.get_cluster_version(), replay_block_header.get_cluster_version());
replay_member_cnt++;
EXPECT_EQ(fill_block_header.flags(), replay_block_header.flags());
replay_member_cnt++;
EXPECT_EQ(replay_member_cnt, fill_member_cnt - 1/*1 skipped*/);
ObTxActiveInfoLogTempRef active_temp_ref;
ObTxActiveInfoLog replay_active_state(active_temp_ref);
@ -564,6 +588,8 @@ TEST_F(TestObTxLog, test_default_log_deserialize)
replay_member_cnt++;
EXPECT_EQ(fill_active_state.get_xid(), replay_active_state.get_xid());
replay_member_cnt++;
EXPECT_EQ(fill_active_state.get_serial_final_seq_no(), replay_active_state.get_serial_final_seq_no());
replay_member_cnt++;
EXPECT_EQ(replay_member_cnt, fill_member_cnt);
ObTxCommitInfoLogTempRef commit_state_temp_ref;
@ -703,10 +729,11 @@ void test_big_commit_info_log(int64_t log_size)
ObTxCommitInfoLog fill_commit_state(TEST_ADDR, TEST_LS_ARRAY, TEST_LS_KEY, TEST_IS_SUB2PC,
TEST_IS_DUP, TEST_CAN_ELR, TEST_TRACE_ID_STR, TEST_TRCE_INFO,
TEST_LOG_OFFSET, TEST_BIG_REDO_LSN_ARRAY, TEST_LS_ARRAY,
TEST_CLUSTER_VERSION, TEST_XID, TEST_COMMIT_PARTS, TEST_EPOCH);
ObTxLogBlockHeader
fill_block_header(TEST_ORG_CLUSTER_ID, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init(TEST_TX_ID, fill_block_header));
TEST_CLUSTER_VERSION, TEST_XID,
TEST_COMMIT_PARTS, TEST_EPOCH);
ObTxLogBlockHeader &fill_block_header = fill_block.get_header();
fill_block_header.init(TEST_ORG_CLUSTER_ID, TEST_CLUSTER_VERSION, TEST_LOG_ENTRY_NO, ObTransID(TEST_TX_ID), TEST_ADDR);
ASSERT_EQ(OB_SUCCESS, fill_block.init_for_fill());
ASSERT_EQ(OB_LOG_TOO_LARGE, fill_block.add_new_log(fill_commit_state, &fill_big_segment));
const char *submit_buf = nullptr;
@ -716,19 +743,17 @@ void test_big_commit_info_log(int64_t log_size)
int ret = OB_SUCCESS;
while (OB_SUCC(ret)
&& OB_EAGAIN
== (ret = (fill_block.acquire_segment_log_buf(submit_buf, submit_buf_len,
fill_block_header,
ObTxLogType::TX_COMMIT_INFO_LOG)))) {
== (ret = (fill_block.acquire_segment_log_buf(ObTxLogType::TX_COMMIT_INFO_LOG)))) {
share::SCN tmp_scn;
EXPECT_EQ(OB_SUCCESS, tmp_scn.convert_for_inner_table_field(part_count));
if (OB_FAIL(fill_block.set_prev_big_segment_scn(tmp_scn))) {
TRANS_LOG(WARN, "set prev big segment scn failed", K(ret), K(part_count));
} else if (OB_FAIL(fill_block.seal(TEST_TX_ID))) {
TRANS_LOG(WARN, "seal block fail", K(ret));
} else {
replay_block.reset();
TxID id = 0;
ObTxLogBlockHeader replay_block_header;
ASSERT_EQ(OB_SUCCESS,
replay_block.init_with_header(submit_buf, submit_buf_len, id, replay_block_header));
ObTxLogBlockHeader &replay_block_header = replay_block.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size()));
if (OB_FAIL(replay_block.get_next_log(log_type_header, &replay_big_segment))) {
TRANS_LOG(WARN, "get next log failed", K(ret), K(part_count));
EXPECT_EQ(OB_LOG_TOO_LARGE, ret);
@ -742,11 +767,10 @@ void test_big_commit_info_log(int64_t log_size)
}
// EXPECT_EQ(ObTxLogType::TX_COMMIT_INFO_LOG, log_type_header.get_tx_log_type());
if (OB_ITER_END == ret) {
fill_block.seal(TEST_TX_ID);
replay_block.reset();
TxID id = 0;
ObTxLogBlockHeader replay_block_header;
ASSERT_EQ(OB_SUCCESS,
replay_block.init_with_header(submit_buf, submit_buf_len, id, replay_block_header));
ObTxLogBlockHeader &replay_block_header = replay_block.get_header();
ASSERT_EQ(OB_SUCCESS, replay_block.init_for_replay(fill_block.get_buf(), fill_block.get_size()));
if (OB_FAIL(replay_block.get_next_log(log_type_header, &replay_big_segment))) {
TRANS_LOG(WARN, "get next log failed", K(ret), K(part_count));
}
@ -790,7 +814,7 @@ void test_big_commit_info_log(int64_t log_size)
EXPECT_EQ(fill_commit_state.get_incremental_participants().count(),
replay_commit_state.get_incremental_participants().count());
replay_member_cnt++;
EXPECT_EQ(fill_commit_state.get_cluster_version(), replay_commit_state.get_cluster_version());
EXPECT_EQ(0, replay_commit_state.get_cluster_version());
replay_member_cnt++;
EXPECT_EQ(fill_commit_state.get_app_trace_id().length(),
replay_commit_state.get_app_trace_id().length());
@ -818,8 +842,115 @@ TEST_F(TestObTxLog, test_big_segment_log)
test_big_commit_info_log(10*1024*1024);
}
} // namespace unittest
TEST_F(TestObTxLog, test_commit_log_with_checksum_signature)
{
uint64_t checksum = 0;
uint8_t sig[64];
ObArrayHelper<uint8_t> checksum_signatures(64, sig);
for(int i = 0; i< 64; i++) {
uint64_t checksum_i = ObRandom::rand(1, 99999);
checksum = ob_crc64(checksum, &checksum_i, sizeof(checksum_i));
checksum_signatures.push_back((uint8_t)(checksum_i & 0xFF));
}
ObLSArray ls_array;
ls_array.push_back(ObLSID(1001));
ObTxBufferNodeArray tx_buffer_node_array;
ObTxBufferNode node;
ObString str("hello,world");
node.init(ObTxDataSourceType::LS_TABLE, str, share::SCN(), nullptr);
tx_buffer_node_array.push_back(node);
ObLSLogInfoArray ls_info_array;
ls_info_array.push_back(ObLSLogInfo());
share::SCN scn;
scn.convert_for_tx(101010101010101);
ObTxCommitLog log0(scn,
checksum,
checksum_signatures,
ls_array,
tx_buffer_node_array,
1,
LogOffSet(100),
ls_info_array);
int64_t size = log0.get_serialize_size();
char *buf = new char[size];
int64_t pos = 0;
ASSERT_EQ(OB_SUCCESS, log0.serialize(buf, size, pos));
ObTxCommitLogTempRef ref;
ObTxCommitLog log1(ref);
pos = 0;
ASSERT_EQ(OB_SUCCESS, log1.deserialize(buf, size, pos));
ASSERT_EQ(log1.checksum_, log0.checksum_);
ASSERT_EQ(log1.checksum_, checksum);
ASSERT_EQ(log1.checksum_sig_.count(), 64);
for(int i = 0; i < log1.checksum_sig_.count(); i++) {
ASSERT_EQ(log1.checksum_sig_.at(i), sig[i]);
}
}
TEST_F(TestObTxLog, test_start_working_log)
{
ObTransID fake_tx_id(0);
ObTxLogBlockHeader header(1, 1, 1, fake_tx_id, ObAddr());
EXPECT_EQ(0, header.get_serialize_size_());
EXPECT_EQ(OB_SUCCESS, header.before_serialize());
int64_t ser_size_ = header.get_serialize_size_();
int64_t ser_size = header.get_serialize_size();
EXPECT_NE(0, ser_size_);
char buf[256];
MEMSET(buf, 0, 256);
int64_t pos = 0;
EXPECT_EQ(OB_SUCCESS, header.serialize(buf, 256, pos));
EXPECT_EQ(pos, ser_size);
ObTxLogBlockHeader header2;
int64_t pos0 = 0;
EXPECT_EQ(OB_SUCCESS, header2.deserialize(buf, 256, pos0));
EXPECT_LE(pos0, pos);
EXPECT_EQ(header2.tx_id_, fake_tx_id);
EXPECT_EQ(header2.cluster_version_, header.cluster_version_);
EXPECT_EQ(header2.log_entry_no_, header.log_entry_no_);
}
TEST_F(TestObTxLog, test_tx_block_header_serialize)
{
// 1. user must call before_serialize, before get_serialize_size(), serialize()
ObTransID tx_id(1024);
ObAddr addr(ObAddr::VER::IPV4, "127.2.3.4", 2048);
ObTxLogBlockHeader header(101, 102, 103, tx_id, addr);
EXPECT_EQ(0, header.serialize_size_);
EXPECT_EQ(OB_SUCCESS, header.before_serialize());
int64_t ser_size_ = header.get_serialize_size_();
EXPECT_EQ(ser_size_, header.get_serialize_size_());
int64_t ser_size = header.get_serialize_size();
EXPECT_GT(ser_size_, 0);
char buf[256];
MEMSET(buf, 0, 256);
int64_t pos = 0;
EXPECT_EQ(OB_SUCCESS, header.serialize(buf, 256, pos));
EXPECT_EQ(pos, ser_size);
// test deserialize ok
ObTxLogBlockHeader header2;
int64_t pos0 = 0;
EXPECT_EQ(OB_SUCCESS, header2.deserialize(buf, 256, pos0));
EXPECT_LE(pos0, pos);
EXPECT_EQ(header2.tx_id_, tx_id);
EXPECT_EQ(header2.org_cluster_id_, 101);
EXPECT_EQ(header2.cluster_version_, 102);
EXPECT_EQ(header2.log_entry_no_, 103);
EXPECT_EQ(header2.scheduler_, addr);
// the serilize size is always equals to header.serialize_size_
header.serialize_size_ = 240;
int64_t ser_size2 = header.get_serialize_size();
EXPECT_GT(ser_size2, 240);
EXPECT_EQ(240, header.get_serialize_size_());
MEMSET(buf, 0, 256);
pos = 0;
EXPECT_EQ(OB_SUCCESS, header.serialize(buf, 256, pos));
EXPECT_EQ(pos, ser_size2);
}
} // namespace unittest
} // namespace oceanbase
using namespace oceanbase;

View File

@ -121,7 +121,7 @@ public:
msg.request_id_ = op_sn_;
msg.savepoint_ = ObTxSEQ(1, 0);
msg.op_sn_ = op_sn_;
msg.branch_id_ = 1;
msg.tx_seq_base_ = 10000000001;
msg.tx_ptr_ = tx;
}
void build_tx_keepalive_msg(ObTxKeepaliveMsg &msg)
@ -432,7 +432,7 @@ TEST_F(TestObTxMsg, trans_rollback_sp_msg)
EXPECT_EQ(msg.cluster_id_, msg1.cluster_id_);
EXPECT_EQ(msg.savepoint_, msg1.savepoint_);
EXPECT_EQ(msg.op_sn_, msg1.op_sn_);
EXPECT_EQ(msg.branch_id_, msg1.branch_id_);
EXPECT_EQ(msg.tx_seq_base_, msg1.tx_seq_base_);
EXPECT_EQ(msg.tx_ptr_->parts_[0].id_, msg1.tx_ptr_->parts_[0].id_);
if (OB_NOT_NULL(msg.tx_ptr_)) {
msg.tx_ptr_ = NULL;

View File

@ -0,0 +1,650 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include <thread>
#define private public
#define protected public
#include "storage/tx/ob_trans_define.h"
#include "storage/tx/ob_trans_service.h"
#include "storage/tx/ob_trans_part_ctx.h"
#include "storage/tx/ob_tx_redo_submitter.h"
#define USING_LOG_PREFIX TRANS
namespace oceanbase
{
using namespace ::testing;
using namespace transaction;
using namespace share;
using namespace memtable;
namespace transaction {
// Test submitter logic works fine in use cases
//
// mock part_ctx's interface:
// - prepare_for_submit_redo
// - fill_log_block
// - submit_log_block_out
// - is_parallel_logging
//
// mock memtable_ctx's interface
// - fill_redo_log
// - log_submitted
//
struct MockDelegate {
virtual bool is_parallel_logging() const = 0;
virtual int submit_redo_log_out(ObTxLogBlock &log_block,
ObTxLogCb *&log_cb,
memtable::ObRedoLogSubmitHelper &helper,
const int64_t replay_hint,
const bool has_hold_ctx_lock,
share::SCN &submitted) = 0;
virtual int fill_redo_log(memtable::ObTxFillRedoCtx &ctx) = 0;
};
struct MockImpl : MockDelegate {
int a_;
public:
MOCK_CONST_METHOD0(is_parallel_logging, bool());
MOCK_METHOD6(submit_redo_log_out, int(ObTxLogBlock &,
ObTxLogCb *&,
memtable::ObRedoLogSubmitHelper &,
const int64_t,
const bool,
share::SCN &));
MOCK_METHOD1(fill_redo_log, int(memtable::ObTxFillRedoCtx &));
};
thread_local MockImpl *mock_ptr;
class ObTestRedoSubmitter : public ::testing::Test
{
public:
virtual void SetUp() override
{
oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION);
ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001);
ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1",2023);
ObCurTraceId::init(ip_port);
GCONF._ob_trans_rpc_timeout = 500;
ObClockGenerator::init();
const testing::TestInfo* const test_info =
testing::UnitTest::GetInstance()->current_test_info();
auto test_name = test_info->name();
_TRANS_LOG(INFO, ">>>> starting test : %s", test_name);
// prepare for test
tx_ctx.exec_info_.state_ = ObTxState::INIT;
tx_ctx.exec_info_.scheduler_ = common::ObAddr(common::ObAddr::VER::IPV4, "127.0.0.1", 8888);
tx_ctx.exec_info_.next_log_entry_no_ = 0;
tx_ctx.cluster_version_ = DATA_CURRENT_VERSION;
ObLSID ls_id(1001); ObTransID tx_id(777);
EXPECT_EQ(OB_SUCCESS,tx_ctx.init_log_cbs_(ls_id, tx_id));
mock_ptr = &mdo_;
}
virtual void TearDown() override
{
const testing::TestInfo* const test_info =
testing::UnitTest::GetInstance()->current_test_info();
auto test_name = test_info->name();
_TRANS_LOG(INFO, ">>>> tearDown test : %s", test_name);
ObClockGenerator::destroy();
ObMallocAllocator::get_instance()->recycle_tenant_allocator(1001);
}
MockImpl mdo_;
ObPartTransCtx tx_ctx;
ObMemtableCtx mt_ctx;
};
int succ_submit_redo_log_out(ObTxLogBlock & b,
ObTxLogCb *& log_cb,
memtable::ObRedoLogSubmitHelper &h,
const int64_t replay_hint,
const bool has_hold_ctx_lock,
share::SCN &submitted_scn)
{
log_cb = NULL;
submitted_scn.convert_for_tx(123123123);
return OB_SUCCESS;
}
bool ObPartTransCtx::is_parallel_logging() const
{
return mock_ptr->is_parallel_logging();
}
int ObPartTransCtx::submit_redo_log_out(ObTxLogBlock &log_block,
ObTxLogCb *&log_cb,
memtable::ObRedoLogSubmitHelper &helper,
const int64_t replay_hint,
const bool has_hold_ctx_lock,
share::SCN &submitted_scn)
{
return mock_ptr->submit_redo_log_out(log_block, log_cb, helper, replay_hint, has_hold_ctx_lock, submitted_scn);
}
}// transaction
namespace memtable {
int ObMemtableCtx::fill_redo_log(ObTxFillRedoCtx &ctx)
{
TRANS_LOG(INFO, "", K(mock_ptr->a_));
return mock_ptr->fill_redo_log(ctx);
}
int ObMemtableCtx::log_submitted(const memtable::ObRedoLogSubmitHelper &helper)
{
TRANS_LOG(INFO, "", K(mock_ptr->a_));
return OB_SUCCESS;
}
int ObMemtableCtx::get_log_guard(const transaction::ObTxSEQ &write_seq,
memtable::ObCallbackListLogGuard &log_guard,
int& cb_list_idx)
{
TRANS_LOG(INFO, "", K(mock_ptr->a_));
cb_list_idx = write_seq.get_branch();
return OB_SUCCESS;
}
} // memtable
namespace transaction {
TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_BLOCK_FROZEN)
{
mdo_.a_ = 1;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(false));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BLOCK_FROZEN;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_BLOCK_FROZEN, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_BUF_NOT_ENOUGH)
{
mdo_.a_ = 2;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(false));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BUF_NOT_ENOUGH;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_ALL_FILLED)
{
mdo_.a_ = 2;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(false));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_SUCCESS;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_UNEXPECTED_ERROR)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(false));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_ALLOCATE_MEMORY_FAILED;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_ALLOCATE_MEMORY_FAILED, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, serial_submit_by_writer_thread_serial_final)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(false));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_SUCCESS;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, true/*serial final*/));
}
}
TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_BLOCK_FROZEN)
{
mdo_.a_ = 4;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BLOCK_FROZEN;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxSEQ writer_seq(101, 0);
EXPECT_EQ(OB_BLOCK_FROZEN, submitter.parallel_submit(writer_seq));
}
}
TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_BLOCKED_BY_OTHER_LIST)
{
mdo_.a_ = 4;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_ITER_END;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxSEQ writer_seq(101, 0);
EXPECT_EQ(OB_ITER_END, submitter.parallel_submit(writer_seq));
}
}
TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_ALL_FILLED)
{
mdo_.a_ = 4;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_SUCCESS;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxSEQ writer_seq(101, 0);
EXPECT_EQ(OB_SUCCESS, submitter.parallel_submit(writer_seq));
}
}
TEST_F(ObTestRedoSubmitter, parallel_submit_by_writer_thread_UNEXPECTED_ERROR)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_ALLOCATE_MEMORY_FAILED;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxSEQ writer_seq(101, 0);
EXPECT_EQ(OB_ALLOCATE_MEMORY_FAILED, submitter.parallel_submit(writer_seq));
}
}
TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_BLOCK_FROZEN)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BLOCK_FROZEN;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_BLOCK_FROZEN, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_BLOCKED_BY_OTHERS)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_ITER_END;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_ITER_END, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_CURRENT_FILLED_BUT_OTHERS_REMAINS)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_EAGAIN;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_EAGAIN, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_BUF_NOT_ENOUGH)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BUF_NOT_ENOUGH;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, submit_by_freeze_parallel_logging_ALL_FILLED)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_SUCCESS;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_SUCCESS, submitter.submit(false/*flush_all*/, false));
}
}
TEST_F(ObTestRedoSubmitter, submit_by_switch_leader_or_on_commit_serial_logging)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(false));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BLOCK_FROZEN;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(1)
.WillOnce(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxLogBlock log_block;
ObTransID tx_id(101);
log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr());
log_block.init_for_fill();
memtable::ObRedoLogSubmitHelper helper;
EXPECT_EQ(OB_BLOCK_FROZEN, submitter.fill(log_block, helper));
EXPECT_EQ(0, helper.callbacks_.count());
}
}
TEST_F(ObTestRedoSubmitter, submit_by_switch_leader_or_on_commit_parallel_logging_ALL_FILLED)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(4)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BLOCK_FROZEN;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 200;
ctx.buf_pos_ = 333;
return OB_ITER_END;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 300;
ctx.buf_pos_ = 444;
return OB_EAGAIN;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 400;
ctx.buf_pos_ = 555;
ObCallbackScope scope;
ctx.helper_->callbacks_.push_back(scope);
return OB_SUCCESS;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(3)
.WillRepeatedly(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxLogBlock log_block;
ObTransID tx_id(101);
log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr());
log_block.init_for_fill();
memtable::ObRedoLogSubmitHelper helper;
EXPECT_EQ(OB_SUCCESS, submitter.fill(log_block, helper));
EXPECT_EQ(helper.callbacks_.count(), 1);
}
}
TEST_F(ObTestRedoSubmitter, submit_by_switch_leader_or_on_commit_parallel_logging_BLOCKED)
{
mdo_.a_ = 3;
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(4)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 100;
ctx.buf_pos_ = 200;
return OB_BLOCK_FROZEN;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 200;
ctx.buf_pos_ = 333;
return OB_ITER_END;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 300;
ctx.buf_pos_ = 444;
return OB_EAGAIN;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 0;
ctx.buf_pos_ = 0;
return OB_BLOCK_FROZEN;
}));
EXPECT_CALL(mdo_, submit_redo_log_out(_,_,_,_,_,_))
.Times(3)
.WillRepeatedly(Invoke(succ_submit_redo_log_out));
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxLogBlock log_block;
ObTransID tx_id(101);
log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr());
log_block.init_for_fill();
memtable::ObRedoLogSubmitHelper helper;
EXPECT_EQ(OB_BLOCK_FROZEN, submitter.fill(log_block, helper));
EXPECT_EQ(helper.callbacks_.count(), 0);
}
}
TEST_F(ObTestRedoSubmitter, submit_ROW_SIZE_TOO_LARGE)
{
EXPECT_CALL(mdo_, is_parallel_logging())
.Times(AtLeast(1))
.WillRepeatedly(Return(true));
{
EXPECT_CALL(mdo_, fill_redo_log(_))
.Times(3)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx) {
ctx.fill_count_ = 0;
ctx.buf_pos_ = 0;
return OB_ERR_TOO_BIG_ROWSIZE;
}));
{
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
EXPECT_EQ(OB_ERR_TOO_BIG_ROWSIZE, submitter.submit(true, false, true));
EXPECT_EQ(submitter.get_submitted_cnt(), 0);
}
{
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxSEQ write_seq(100,200);
EXPECT_EQ(OB_ERR_TOO_BIG_ROWSIZE, submitter.parallel_submit(write_seq));
EXPECT_EQ(submitter.get_submitted_cnt(), 0);
}
{
ObTxRedoSubmitter submitter(tx_ctx, mt_ctx);
ObTxLogBlock log_block;
ObTransID tx_id(101);
log_block.get_header().init(1, DATA_CURRENT_VERSION, 101, tx_id, ObAddr());
log_block.init_for_fill();
memtable::ObRedoLogSubmitHelper helper;
EXPECT_EQ(OB_ERR_TOO_BIG_ROWSIZE, submitter.fill(log_block, helper));
EXPECT_EQ(submitter.get_submitted_cnt(), 0);
EXPECT_EQ(helper.callbacks_.count(), 0);
}
}
}
} // transaction
} // oceanbase
int main(int argc, char **argv)
{
const char *log_name = "test_redo_submitter.log";
system("rm -rf test_redo_submitter.log*");
ObLogger &logger = ObLogger::get_logger();
logger.set_file_name(log_name, true, false,
log_name,
log_name,
log_name);
logger.set_log_level(OB_LOG_LEVEL_DEBUG);
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View File

@ -0,0 +1,774 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include <thread>
#define private public
#define protected public
#include "storage/tx/ob_trans_define.h"
#include "storage/tx/ob_trans_service.h"
#include "storage/tx/ob_trans_part_ctx.h"
#include "storage/tx/ob_tx_redo_submitter.h"
#define USING_LOG_PREFIX TRANS
namespace oceanbase
{
using namespace ::testing;
using namespace transaction;
using namespace share;
namespace memtable {
// Test fill redo logic works for multi-callback-list
//
// target interface to test:
// - fill_from_one_list
// - fill_from_all_list
//
// target situation:
// - BLOCK_FROZEN
// - blocked by other list
// - all callback filled successfully
//
// logging mode:
// - parallel logging
// - serial logging
//
// mock callback list interface
// - fill_log
// - get_log_epoch
//
struct MockDelegate {
virtual int fill_log(ObTxFillRedoCtx &fill_ctx) = 0;
virtual int64_t get_log_epoch(int i) const = 0;
virtual int get_logging_list_count() const = 0;
};
struct MockImpl : MockDelegate {
struct MockCallback : public ObITransCallback
{
};
int a_;
ObTxFillRedoCtx fill_ctx_;
MockCallback cb1_;
MockCallback cb2_;
MockCallback cb3_;
MockImpl() : cb1_() {
cb1_.next_ = &cb2_;
cb1_.prev_ = &cb3_;
cb2_.next_ = &cb3_;
cb2_.prev_ = &cb1_;
cb3_.next_ = &cb1_;
cb3_.prev_ = &cb2_;
}
void init_callback_scope()
{
fill_ctx_.callback_scope_->start_.cur_ = &cb1_;
fill_ctx_.callback_scope_->end_.cur_ = &cb2_;
}
public:
MOCK_CONST_METHOD0(get_logging_list_count, int());
MOCK_CONST_METHOD1(get_log_epoch, int64_t(int));
MOCK_METHOD1(fill_log, int(ObTxFillRedoCtx&));
};
thread_local MockImpl *mock_ptr;
int ObTxCallbackList::fill_log(ObITransCallback* cursor, ObTxFillRedoCtx &ctx, ObITxFillRedoFunctor &func)
{
mock_ptr->init_callback_scope();
return mock_ptr->fill_log(ctx);
}
int64_t ObTxCallbackList::get_log_epoch() const
{
return mock_ptr->get_log_epoch(id_);
}
int ObTransCallbackMgr::get_logging_list_count() const
{
return mock_ptr->get_logging_list_count();
}
class ObTestRedoFill : public ::testing::Test
{
public:
ObTestRedoFill():
mem_ctx_(),
cb_allocator_(),
callback_mgr_(mem_ctx_, cb_allocator_) {}
virtual void SetUp() override
{
oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION);
ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(1001);
ObAddr ip_port(ObAddr::VER::IPV4, "119.119.0.1",2023);
ObCurTraceId::init(ip_port);
GCONF._ob_trans_rpc_timeout = 500;
ObClockGenerator::init();
const testing::TestInfo* const test_info =
testing::UnitTest::GetInstance()->current_test_info();
auto test_name = test_info->name();
_TRANS_LOG(INFO, ">>>> starting test : %s", test_name);
mdo_.fill_ctx_.helper_ = &helper_;
mock_ptr = &mdo_;
cb_allocator_.init(1001);
}
virtual void TearDown() override
{
const testing::TestInfo* const test_info =
testing::UnitTest::GetInstance()->current_test_info();
auto test_name = test_info->name();
_TRANS_LOG(INFO, ">>>> tearDown test : %s", test_name);
ObClockGenerator::destroy();
ObMallocAllocator::get_instance()->recycle_tenant_allocator(1001);
}
void extend_callback_lists_(int cnt)
{
callback_mgr_.callback_lists_ = (ObTxCallbackList*)new char[(sizeof(ObTxCallbackList) * cnt)];
for (int i=0; i<cnt; i++) {
new (callback_mgr_.callback_lists_ + i) ObTxCallbackList(callback_mgr_, i + 1);
}
}
void set_parallel_logging(bool t)
{
if (t) {
share::SCN scn;
scn.convert_for_tx(1231231231);
callback_mgr_.set_parallel_logging(scn);
} else {
callback_mgr_.set_parallel_logging(share::SCN::max_scn());
}
}
ObRedoLogSubmitHelper helper_;
ObMemtableCtx mem_ctx_;
ObMemtableCtxCbAllocator cb_allocator_;
ObTransCallbackMgr callback_mgr_;
MockImpl mdo_;
class ObTxFillRedoFunctor : public ObITxFillRedoFunctor {
int operator()(ObITransCallback * cb) { return OB_SUCCESS; }
} fill_func;
};
TEST_F(ObTestRedoFill, serial_single_list_fill_all_BLOCK_FROZEN)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// single list
callback_mgr_.callback_lists_ = NULL;
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(1));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 999;
return OB_BLOCK_FROZEN;
}));
EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 999);
}
TEST_F(ObTestRedoFill, serial_multi_list_fill_all_ALL_FILLED)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100, 97, 98, 101};
return epochs[i];
}));
int i = 0;
{
InSequence s;
// fill orders: init: {100,97,98,101}
// -> {100,100,98,101}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 2: 97 -> 100
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 97;
ctx.next_epoch_ = 100;
return OB_ITER_END;
}));
++i;
// -> {100,100,101,101}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 3: 98 -> 101
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 98;
ctx.next_epoch_ = 101;
return OB_ITER_END;
}));
++i;
// -> {102,100,101,101}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 1: 100 -> 102
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 100;
ctx.next_epoch_ = 102;
return OB_ITER_END;
}));
++i;
// -> {102,102,101,101}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 2: 101 -> 102
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 100;
ctx.next_epoch_ = 102;
return OB_ITER_END;
}));
++i;
// -> {102,102,END,101}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 3: 101 -> END
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 101;
return OB_SUCCESS;
}));
++i;
// -> {102,102,END,END}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 4: 101 -> END
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 101;
return OB_SUCCESS;
}));
++i;
// -> {END,102,END,END}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 1: 102 -> END
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 102;
return OB_SUCCESS;
}));
++i;
// -> {END,END,END,END}
EXPECT_CALL(mdo_, fill_log(_)).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
// list 2: 102 -> END
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 102;
return OB_SUCCESS;
}));
++i;
}
auto &ctx = mdo_.fill_ctx_;
EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(ctx, fill_func));
EXPECT_EQ(ctx.epoch_from_, 102);
EXPECT_EQ(ctx.epoch_to_, 102);
EXPECT_EQ(ctx.fill_count_, i);
EXPECT_EQ(ctx.list_log_epoch_arr_[0], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[1], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[2], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[3], INT64_MAX);
}
TEST_F(ObTestRedoFill, serial_multi_list_fill_all_BLOCK_FROZEN)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100, 100, 100, 100};
return epochs[i];
}));
{
InSequence s;
// list 1,2,3 blocked
EXPECT_CALL(mdo_, fill_log(_)).Times(3)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 100;
return OB_BLOCK_FROZEN;
}));
// list 4 filled
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 10;
ctx.cur_epoch_ = 100;
return OB_SUCCESS;
}));
// next round
// fill list 1,2,3 again success
EXPECT_CALL(mdo_, fill_log(_)).Times(3)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 100;
return OB_SUCCESS;
}));
}
auto &ctx = mdo_.fill_ctx_;
EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(ctx, fill_func));
EXPECT_EQ(ctx.epoch_from_, 100);
EXPECT_EQ(ctx.epoch_to_, 100);
EXPECT_EQ(ctx.fill_count_, 313);
EXPECT_EQ(ctx.list_log_epoch_arr_[0], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[1], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[2], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[3], INT64_MAX);
}
TEST_F(ObTestRedoFill, serial_single_list_fill_all_BUF_NOT_ENOUGH)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// single list
callback_mgr_.callback_lists_ = NULL;
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(1));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 999;
return OB_BUF_NOT_ENOUGH;
}));
EXPECT_EQ(OB_BUF_NOT_ENOUGH, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 999);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 999);
}
TEST_F(ObTestRedoFill, serial_single_list_fill_all_list_BIG_ROW)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// single list
callback_mgr_.callback_lists_ = NULL;
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(1));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 0;
ctx.cur_epoch_ = 100;
return OB_BUF_NOT_ENOUGH;
}));
EXPECT_EQ(OB_BUF_NOT_ENOUGH, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 100);
}
TEST_F(ObTestRedoFill, serial_multi_list_fill_all_list_BIG_ROW)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,99,100,99};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 0;
ctx.cur_epoch_ = 99;
return OB_BUF_NOT_ENOUGH;
}));
EXPECT_EQ(OB_BUF_NOT_ENOUGH, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 99);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 1);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 100);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 99);
}
TEST_F(ObTestRedoFill, serial_multi_list_fill_all_OTHER_ERROR_WHEN_FILL_OTHERS)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100, 100, 100, 100};
return epochs[i];
}));
{
InSequence s;
// list 1 ALL FILLED
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 100;
return OB_SUCCESS;
}));
// list 2 FROZEN BLOCKED
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 10;
ctx.cur_epoch_ = 100;
return OB_BLOCK_FROZEN;
}));
// list 3 MEMORY_ALLOCATE_FAILED
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 100;
return OB_ALLOCATE_MEMORY_FAILED;
}));
}
auto &ctx = mdo_.fill_ctx_;
EXPECT_EQ(OB_ALLOCATE_MEMORY_FAILED, callback_mgr_.fill_from_all_list(ctx, fill_func));
EXPECT_EQ(ctx.epoch_from_, 100);
EXPECT_EQ(ctx.epoch_to_, 100);
EXPECT_EQ(ctx.fill_count_, 111);
EXPECT_EQ(ctx.list_log_epoch_arr_[0], INT64_MAX);
EXPECT_EQ(ctx.list_log_epoch_arr_[1], 100);
EXPECT_EQ(ctx.list_log_epoch_arr_[2], 100);
EXPECT_EQ(ctx.list_log_epoch_arr_[3], 100);
}
TEST_F(ObTestRedoFill, serial_logging_fill_from_all_list_ALL_OTHERS_BLOCK_FROZEN_EMPTY)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,100,100,100};
return epochs[i];
}));
{
InSequence s;
// list 1 ALL FILLED
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 100;
return OB_SUCCESS;
}));
// list 2,3,4 BLOCK_FROZEN AND NOTHING FILLED
EXPECT_CALL(mdo_, fill_log(_)).Times(3)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 0;
ctx.cur_epoch_ = 100;
return OB_BLOCK_FROZEN;
}));
}
EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 100);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100);
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 1);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 3); // point to list 4
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], INT64_MAX);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 100);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 100);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[3], 100);
}
TEST_F(ObTestRedoFill, serial_logging_fill_from_all_list_FIRST_ITER_END_OTHERS_BLOCK_FROZEN_EMPTY)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,101,100,200};
return epochs[i];
}));
{
InSequence s;
// list 1 ITER END
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 1;
ctx.cur_epoch_ = 100;
ctx.next_epoch_ = 101;
return OB_ITER_END;
}));
// list 2 skipped
// list 3, BLOCK_FROZEN AND NOTHING FILLED
EXPECT_CALL(mdo_, fill_log(_)).Times(1)
.WillRepeatedly(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 0;
ctx.cur_epoch_ = 100;
return OB_BLOCK_FROZEN;
}));
// list 4 skipped
}
EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 100);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100);
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 1);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2); // point to list 3
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], 101);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 101);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 100);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[3], 200);
}
TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_FROZEN)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,99,98,99};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 99;
return OB_BLOCK_FROZEN;
}));
EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99);
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 99);
}
TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_FROZEN_EMPTY)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,99,98,99};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 0;
ctx.cur_epoch_ = 98;
return OB_BLOCK_FROZEN;
}));
EXPECT_EQ(OB_BLOCK_FROZEN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 98);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2);
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 0);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 98);
}
TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_FROZEN_FILL_FROM_OTHERS)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,98,98,99};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(2)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 0;
ctx.cur_epoch_ = 98;
return OB_BLOCK_FROZEN;
}))
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 99;
ctx.next_epoch_ = 101;
return OB_ITER_END;
}));
EXPECT_EQ(OB_ITER_END, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 98);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2);
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[1], 98);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 101);
}
TEST_F(ObTestRedoFill, parallel_multi_list_fill_all_BLOCK_BY_OTHERS)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,99,98,99};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 98;
ctx.next_epoch_ = 101;
return OB_ITER_END;
}));
EXPECT_EQ(OB_ITER_END, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99);
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], 101);
}
TEST_F(ObTestRedoFill, parallel_logging_fill_from_all_list_ALL_FILLED_OTHERS_REMAIN)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,99,98,99};
return epochs[i];
}));
EXPECT_CALL(mdo_, fill_log(_))
.Times(1)
.WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 100;
ctx.cur_epoch_ = 98;
return OB_SUCCESS;
}));
EXPECT_EQ(OB_EAGAIN, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 98);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, 99);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 99); // consume done, the cur_epoch point to epoch_to
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 100);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 2);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[2], INT64_MAX);
}
TEST_F(ObTestRedoFill, parallel_logging_fill_from_one_list_OTHERS_IS_EMPTY)
{
set_parallel_logging(true);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,INT64_MAX,INT64_MAX,INT64_MAX};
return epochs[i];
}));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_log(_)).Times(1).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 123;
ctx.cur_epoch_ = 100;
return OB_SUCCESS;
}));
}
EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); // consume done, the cur_epoch point to epoch_to
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 123);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 0);
EXPECT_TRUE(mdo_.fill_ctx_.is_all_filled_);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], INT64_MAX);
}
TEST_F(ObTestRedoFill, serial_logging_fill_from_one_list_OTHERS_IS_EMPTY)
{
set_parallel_logging(false);
callback_mgr_.need_merge_ = false;
// 4 list
extend_callback_lists_(3);
EXPECT_CALL(mdo_, get_logging_list_count()).Times(AtLeast(1)).WillRepeatedly(Return(4));
EXPECT_CALL(mdo_, get_log_epoch(_))
.Times(AtLeast(1))
.WillRepeatedly(Invoke([](int i){
int64_t epochs[] = {100,INT64_MAX,INT64_MAX,INT64_MAX};
return epochs[i];
}));
{
InSequence s1;
EXPECT_CALL(mdo_, fill_log(_)).Times(1).WillOnce(Invoke([](ObTxFillRedoCtx &ctx){
ctx.fill_count_ += 123;
ctx.cur_epoch_ = 100;
return OB_SUCCESS;
}));
}
EXPECT_EQ(OB_SUCCESS, callback_mgr_.fill_from_all_list(mdo_.fill_ctx_, fill_func));
EXPECT_EQ(mdo_.fill_ctx_.epoch_from_, 100);
EXPECT_EQ(mdo_.fill_ctx_.epoch_to_, INT64_MAX);
EXPECT_EQ(mdo_.fill_ctx_.cur_epoch_, 100); // consume done, the cur_epoch point to epoch_to
EXPECT_EQ(mdo_.fill_ctx_.fill_count_, 123);
EXPECT_EQ(mdo_.fill_ctx_.list_idx_, 0);
EXPECT_TRUE(mdo_.fill_ctx_.is_all_filled_);
EXPECT_EQ(mdo_.fill_ctx_.list_log_epoch_arr_[0], INT64_MAX);
}
} // memtable
} // oceanbase
int main(int argc, char **argv)
{
const char *log_name = "test_trans_callback_mgr_fill_redo.log";
system("rm -rf test_trans_callback_mgr_fill_redo.log*");
ObLogger &logger = ObLogger::get_logger();
logger.set_file_name(log_name, true, false,
log_name,
log_name,
log_name);
logger.set_log_level(OB_LOG_LEVEL_DEBUG);
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View File

@ -0,0 +1,86 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include "storage/tx/ob_trans_define.h"
#include <gtest/gtest.h>
#include "lib/oblog/ob_log.h"
namespace oceanbase
{
using namespace transaction;
namespace unittest
{
struct TestUndoAction : public ::testing::Test
{
virtual void SetUp() {}
virtual void TearDown() {}
};
TEST_F(TestUndoAction, valid)
{
ObUndoAction a1(ObTxSEQ(100, 0), ObTxSEQ(1, 1));
EXPECT_FALSE(a1.is_valid());
ObUndoAction a2(ObTxSEQ(100, 1), ObTxSEQ(100, 1));
EXPECT_FALSE(a2.is_valid());
ObUndoAction a3(ObTxSEQ(100, 0), ObTxSEQ(100, 0));
EXPECT_FALSE(a3.is_valid());
ObUndoAction a4(ObTxSEQ(100, 0), ObTxSEQ(100, 1));
EXPECT_FALSE(a4.is_valid());
ObUndoAction a5(ObTxSEQ(100, 1), ObTxSEQ(100, 0));
EXPECT_FALSE(a5.is_valid());
ObUndoAction a6(ObTxSEQ(100, 1), ObTxSEQ(1, 0));
EXPECT_FALSE(a6.is_valid());
ObUndoAction a7(ObTxSEQ(100, 1), ObTxSEQ(1, 1));
EXPECT_TRUE(a7.is_valid());
ObUndoAction a8(ObTxSEQ(100, 0), ObTxSEQ(1, 0));
EXPECT_TRUE(a8.is_valid());
}
TEST_F(TestUndoAction, contain)
{
ObUndoAction a1(ObTxSEQ(100,1), ObTxSEQ(1, 1));
ObUndoAction a2(ObTxSEQ(99,1), ObTxSEQ(1, 1));
EXPECT_TRUE(a1.is_contain(a2));
EXPECT_FALSE(a2.is_contain(a1));
ObUndoAction a3(ObTxSEQ(100,0), ObTxSEQ(1, 0));
ObUndoAction a4(ObTxSEQ(99,0), ObTxSEQ(1, 0));
EXPECT_TRUE(a3.is_contain(a4));
EXPECT_FALSE(a4.is_contain(a3));
ObUndoAction a5(ObTxSEQ(100,2), ObTxSEQ(1, 2));
EXPECT_FALSE(a5.is_contain(a1));
EXPECT_FALSE(a5.is_contain(a2));
EXPECT_FALSE(a1.is_contain(a5));
EXPECT_TRUE(a3.is_contain(a5));
EXPECT_FALSE(a4.is_contain(a5));
}
TEST_F(TestUndoAction, contain_point)
{
ObUndoAction a1(ObTxSEQ(100,1), ObTxSEQ(1, 1));
ObUndoAction a3(ObTxSEQ(100,0), ObTxSEQ(1, 0));
EXPECT_TRUE(a3.is_contain(ObTxSEQ(50, 1)));
EXPECT_TRUE(a1.is_contain(ObTxSEQ(50, 1)));
EXPECT_FALSE(a1.is_contain(ObTxSEQ(50, 0)));
EXPECT_FALSE(a1.is_contain(ObTxSEQ(50, 2)));
EXPECT_TRUE(a3.is_contain(ObTxSEQ(50, 0)));
}
} // unittest
} //oceanbase
using namespace oceanbase;
using namespace transaction;
int main(int argc, char **argv)
{
int ret = 1;
ObLogger &logger = ObLogger::get_logger();
logger.set_file_name("test_undo_action.log", true);
logger.set_log_level(OB_LOG_LEVEL_INFO);
testing::InitGoogleTest(&argc, argv);
ret = RUN_ALL_TESTS();
return ret;
}

View File

@ -248,6 +248,7 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr)
context.store_ctx_ = &store_ctx;
context.allocator_ = &allocator;
context.stmt_allocator_ = &allocator;
context.merge_scn_.convert_from_ts(996);
context.is_inited_ = true;
ObDatumRange key_range;
@ -274,6 +275,8 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr)
ctx1.is_inited_ = true;
ctx1.ls_id_ = ls_id;
ctx1.exec_info_.max_applying_log_ts_.convert_from_ts(1);
ctx1.replay_completeness_.set(true);
ctx1.rec_log_ts_.convert_from_ts(996);
ObTxData data1;
// ctx1.tx_data_ = &data1;
ctx1.ctx_tx_data_.test_init(data1, &ls_tx_ctx_mgr_);
@ -285,6 +288,8 @@ TEST_F(TestTxCtxTable, test_tx_ctx_memtable_mgr)
ctx2.is_inited_ = true;
ctx2.ls_id_ = ls_id;
ctx2.exec_info_.max_applying_log_ts_.convert_from_ts(2);
ctx2.replay_completeness_.set(true);
ctx2.rec_log_ts_.convert_from_ts(996);
ObTxData data2;
// ctx2.tx_data_ = &data2;
ctx2.ctx_tx_data_.test_init(data2, &ls_tx_ctx_mgr_);