init push

This commit is contained in:
oceanbase-admin
2021-05-31 22:56:52 +08:00
commit cea7de1475
7020 changed files with 5689869 additions and 0 deletions

View File

@ -0,0 +1,185 @@
/**
* 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_UNITTEST_CLOG_MOCK_OB_ELECTION_MGR_H
#define OCEANBASE_UNITTEST_CLOG_MOCK_OB_ELECTION_MGR_H
#include "election/ob_election_mgr.h"
#include "election/ob_election_base.h"
#include "election/ob_election_rpc.h"
#include "common/ob_partition_key.h"
#include "common/ob_member_list.h"
#include "lib/hash/ob_hashmap.h"
#include "lib/lock/ob_spin_rwlock.h"
#include "storage/transaction/ob_time_wheel.h"
namespace oceanbase {
namespace common {
class ObPartitionKey;
class ObMemberList;
class ObAddr;
} // namespace common
namespace obrpc {
class ObElectionRpcResult;
class ObElectionRpcProxy;
} // namespace obrpc
namespace election {
class ObIElection;
class ObElectionMsgBuffer;
class ObIElectionCallback;
} // namespace election
namespace election {
class MockObElectionMgr : public ObElectionMgr {
public:
MockObElectionMgr()
{}
~MockObElectionMgr()
{
destroy();
}
int init(const common::ObAddr& self, obrpc::ObElectionRpcProxy* rpc_proxy)
{
UNUSED(self);
UNUSED(rpc_proxy);
return OB_SUCCESS;
}
void destroy()
{}
int start_partition(const common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
int start_partition(
const common::ObPartitionKey& partition_key, const common::ObAddr& leader, const int64_t& lease_start)
{
UNUSED(partition_key);
UNUSED(leader);
UNUSED(lease_start);
return OB_SUCCESS;
}
int stop_partition(const common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
int start()
{
return OB_SUCCESS;
}
int wait()
{
return OB_SUCCESS;
}
int stop()
{
return OB_SUCCESS;
}
public:
int add_partition(
const common::ObPartitionKey& partition_key, const int64_t replica_num, ObIElectionCallback* election_cb)
{
UNUSED(partition_key);
UNUSED(replica_num);
UNUSED(election_cb);
return OB_SUCCESS;
}
int remove_partition(const common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
int change_leader_async(const common::ObPartitionKey& partition_key, const common::ObAddr& leader)
{
UNUSED(partition_key);
UNUSED(leader);
return OB_SUCCESS;
}
int leader_revoke(const common::ObPartitionKey& partition)
{
UNUSED(partition);
return OB_SUCCESS;
}
public:
int handle_election_msg(const ObElectionMsgBuffer& msgbuf, obrpc::ObElectionRpcResult& result)
{
UNUSED(msgbuf);
UNUSED(result);
return OB_SUCCESS;
}
public:
int set_candidate(const common::ObPartitionKey& partition_key, const common::ObMemberList& prev_mlist,
const common::ObMemberList& curr_mlist)
{
UNUSED(partition_key);
UNUSED(prev_mlist);
UNUSED(curr_mlist);
return OB_SUCCESS;
}
int get_prev_candidate(const common::ObPartitionKey& partition_key, common::ObMemberList& mlist) const
{
UNUSED(partition_key);
UNUSED(mlist);
return OB_SUCCESS;
}
int get_curr_candidate(const common::ObPartitionKey& partition_key, common::ObMemberList& mlist) const
{
UNUSED(partition_key);
UNUSED(mlist);
return OB_SUCCESS;
}
int get_leader(const common::ObPartitionKey& partition_key, common::ObAddr& leader) const
{
UNUSED(partition_key);
UNUSED(leader);
return OB_SUCCESS;
}
int get_leader(const common::ObPartitionKey& partition_key, common::ObAddr& leader, int64_t& leader_epoch,
bool& is_changing_leader) const
{
UNUSED(partition_key);
UNUSED(leader);
UNUSED(leader_epoch);
UNUSED(is_changing_leader);
return OB_SUCCESS;
}
private:
ObIElection* get_election_(const common::ObPartitionKey& partition_key) const
{
UNUSED(partition_key);
return NULL;
}
int stop_()
{
return OB_SUCCESS;
}
int wait_()
{
return OB_SUCCESS;
}
};
} // namespace election
} // namespace oceanbase
#endif // OCEANBASE_UNITTEST_CLOG_MOCK_OB_ELECTION_MGR_H

View File

@ -0,0 +1,18 @@
/**
* 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 MOCK_OB_LOG_ALLOCATOR_H
#define MOCK_OB_LOG_ALLOCATOR_H
namespace oceanbase {
namespace clog {} // namespace clog
} // namespace oceanbase
#endif // MOCK_OB_LOG_ALLOCATOR_H

View File

@ -0,0 +1,593 @@
/**
* 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_UNITTEST_CLOG_MOCK_CONTAINER_MOCK_OB_LOG_ENGINE_H_
#define OOCEANBASE_UNITTEST_CLOG_MOCK_CONTAINER_MOCK_OB_LOG_ENGINE_H_
#include "clog/ob_i_disk_log_buffer.h"
#include "common/ob_member_list.h"
#include "lib/net/ob_addr.h"
#include "common/ob_partition_key.h"
#include "clog/ob_log_reader_interface.h"
#include "clog/ob_i_log_engine.h"
namespace oceanbase {
namespace clog {
class ObIDiskLogBuffer;
class ObINetLogBufferMgr;
class ObILogNetTask;
class ObConfirmedInfo;
class ObLogTask;
class MockObLogEngine : public ObILogEngine {
public:
typedef ObDiskBufferTask FlushTask;
public:
MockObLogEngine()
{}
~MockObLogEngine()
{}
ObIRawLogIterator* alloc_raw_log_iterator(
const file_id_t start_file_id, const file_id_t end_file_id, const offset_t offset, const int64_t timeout)
{
UNUSED(start_file_id);
UNUSED(end_file_id);
UNUSED(offset);
UNUSED(timeout);
return NULL;
}
void revert_raw_log_iterator(ObIRawLogIterator* iter)
{
UNUSED(iter);
}
ObIRawIndexIterator* alloc_raw_index_iterator(const file_id_t start_file_id, const file_id_t end_file_id,
const offset_t offset, const int64_t timeout, const bool use_cache)
{
UNUSED(start_file_id);
UNUSED(end_file_id);
UNUSED(offset);
UNUSED(timeout);
UNUSED(use_cache);
return NULL;
}
void revert_raw_index_iterator(ObIRawIndexIterator* iter)
{
UNUSED(iter);
}
int read_log_by_location(const ObReadParam& param, ObLogEntry& entry)
{
UNUSED(param);
UNUSED(entry);
return common::OB_SUCCESS;
}
int read_log_by_location(const ObReadParam& param, ObLogEntry& entry, ObReadCost& cost)
{
UNUSED(param);
UNUSED(entry);
UNUSED(cost);
return common::OB_SUCCESS;
}
int read_log_by_location(const ObLogTask& log_task, ObLogEntry& entry)
{
UNUSED(log_task);
UNUSED(entry);
return common::OB_SUCCESS;
}
int read_data_with_ctx(ObReadCtx& ctx, file_id_t f, offset_t o, int64_t s, char* b, ObReadCost& cost)
{
UNUSED(ctx);
UNUSED(f);
UNUSED(o);
UNUSED(s);
UNUSED(b);
UNUSED(cost);
return common::OB_SUCCESS;
}
int read_data_from_hot_cache(
const file_id_t want_file_id, const offset_t want_offset, const int64_t want_size, char* user_buf)
{
UNUSED(want_file_id);
UNUSED(want_offset);
UNUSED(want_size);
UNUSED(user_buf);
return common::OB_SUCCESS;
}
int read_data_direct(const ObReadParam& param, ObReadBuf& rbuf, ObReadRes& res, ObReadCost& cost)
{
UNUSED(param);
UNUSED(rbuf);
UNUSED(res);
UNUSED(cost);
return common::OB_SUCCESS;
}
int submit_flush_task(FlushTask* task)
{
UNUSED(task);
return common::OB_SUCCESS;
}
int submit_fetch_log_resp(
const common::ObAddr& server, const common::ObPartitionKey& key, const int64_t network_limit, ObILogNetTask* task)
{
UNUSED(server);
UNUSED(key);
UNUSED(network_limit);
UNUSED(task);
return common::OB_SUCCESS;
}
int submit_log_ack(const common::ObAddr& server, const common::ObPartitionKey& key, const uint64_t log_id,
const common::ObProposalID proposal_id)
{
UNUSED(server);
UNUSED(key);
UNUSED(log_id);
UNUSED(proposal_id);
return common::OB_SUCCESS;
}
int fetch_log_from_all_follower(const common::ObMemberList& mem_list, const common::ObPartitionKey& key,
const uint64_t start_id, const uint64_t end_id, const common::ObProposalID proposal_id)
{
UNUSED(mem_list);
UNUSED(key);
UNUSED(start_id);
UNUSED(end_id);
UNUSED(proposal_id);
return common::OB_SUCCESS;
}
int fetch_log_from_leader(const common::ObAddr& server, const common::ObPartitionKey& key, const uint64_t start_id,
const uint64_t end_id, const common::ObProposalID proposal_id, const common::ObReplicaType replica_type)
{
UNUSED(server);
UNUSED(key);
UNUSED(start_id);
UNUSED(end_id);
UNUSED(proposal_id);
UNUSED(replica_type);
return common::OB_SUCCESS;
}
int fetch_register_server(const common::ObAddr& server, const ObPartitionKey& key,
const common::ObReplicaType replica_type, const bool is_request_leader, const bool is_need_force_register)
{
UNUSED(server);
UNUSED(key);
UNUSED(replica_type);
UNUSED(is_request_leader);
UNUSED(is_need_force_register);
return common::OB_SUCCESS;
}
int request_replace_sick_child(
const common::ObAddr& server, const common::ObPartitionKey& key, const common::ObAddr& sick_child)
{
int ret = OB_SUCCESS;
UNUSED(server);
UNUSED(key);
UNUSED(sick_child);
return ret;
}
int reject_server(const common::ObAddr& server, const common::ObPartitionKey& key, const int32_t msg_type)
{
UNUSED(server);
UNUSED(key);
UNUSED(msg_type);
return common::OB_SUCCESS;
}
bool is_in_black_list(const common::ObAddr& server) const
{
UNUSED(server);
return true;
}
int submit_prepare_rqst(
const common::ObMemberList& mem_list, const common::ObPartitionKey& key, const common::ObProposalID proposal_id)
{
UNUSED(mem_list);
UNUSED(key);
UNUSED(proposal_id);
return common::OB_SUCCESS;
}
int prepare_response(const common::ObAddr& server, const common::ObPartitionKey& partition_key,
const common::ObProposalID proposal_id, const uint64_t max_log_id, const int64_t max_log_ts)
{
UNUSED(server);
UNUSED(partition_key);
UNUSED(proposal_id);
UNUSED(max_log_id);
UNUSED(max_log_ts);
return common::OB_SUCCESS;
}
int send_keepalive_msg(const common::ObAddr& server, const common::ObPartitionKey& partition_key,
const uint64_t next_log_id, const int64_t next_log_ts_lb, const uint64_t deliver_cnt)
{
UNUSED(server);
UNUSED(partition_key);
UNUSED(next_log_id);
UNUSED(next_log_ts_lb);
UNUSED(deliver_cnt);
return common::OB_SUCCESS;
}
int send_sync_log_archive_progress_msg(const common::ObAddr& server, const int64_t cluster_id,
const common::ObPartitionKey& partition_key, const int64_t log_archive_round, const uint64_t last_archived_log_id,
const int64_t last_archived_log_ts)
{
UNUSED(server);
UNUSED(cluster_id);
UNUSED(partition_key);
UNUSED(log_archive_round);
UNUSED(last_archived_log_id);
UNUSED(last_archived_log_ts);
return common::OB_SUCCESS;
}
int submit_index_flush_task(FlushTask* task)
{
UNUSED(task);
return common::OB_SUCCESS;
}
void update_clog_info(const int64_t submit_timestamp)
{
UNUSED(submit_timestamp);
}
void update_clog_info(
const common::ObPartitionKey& partition_key, const uint64_t log_id, const int64_t submit_timestamp)
{
UNUSED(partition_key);
UNUSED(log_id);
UNUSED(submit_timestamp);
}
void update_ilog_info(
const common::ObPartitionKey& partition_key, const uint64_t log_id, const int64_t submit_timestamp)
{
UNUSED(partition_key);
UNUSED(log_id);
UNUSED(submit_timestamp);
}
void update_ilog_info(const int64_t submit_timestamp)
{
UNUSED(submit_timestamp);
}
ObILogInfoBlockReader* get_info_block_reader(const InfoBlockReaderType type)
{
UNUSED(type);
return NULL;
}
void revert_info_block_reader(ObILogInfoBlockReader* reader)
{
UNUSED(reader);
}
// new functions
ObILogInfoBlockReader* get_clog_info_block_reader()
{
return NULL;
}
void revert_clog_info_block_reader(ObILogInfoBlockReader* reader)
{
UNUSED(reader);
}
ObILogInfoBlockReader* get_ilog_info_block_reader()
{
return NULL;
}
void revert_ilog_info_block_reader(ObILogInfoBlockReader* reader)
{
UNUSED(reader);
}
virtual ObCommitLogEnv* get_clog_env(const bool is_sys)
{
UNUSED(is_sys);
return NULL;
}
virtual ObIndexLogEnv* get_ilog_env(const bool is_sys)
{
UNUSED(is_sys);
return NULL;
}
virtual void revert_clog_env(ObCommitLogEnv* log_env)
{
UNUSED(log_env);
}
virtual void revert_ilog_env(ObIndexLogEnv* log_env)
{
UNUSED(log_env);
}
int reset_clog_info_block()
{
return common::OB_SUCCESS;
}
int reset_ilog_info_block()
{
return common::OB_SUCCESS;
}
int switch_ilog_file()
{
return common::OB_SUCCESS;
}
virtual int notify_follower_log_missing(
const common::ObAddr& a, const common::ObPartitionKey& b, uint64_t c, const bool d)
{
UNUSED(a);
UNUSED(b);
UNUSED(c);
UNUSED(d);
return common::OB_SUCCESS;
}
virtual file_id_t get_flying_ilog_file_id()
{
return common::OB_INVALID_FILE_ID;
}
virtual file_id_t get_flying_clog_file_id()
{
return common::OB_INVALID_FILE_ID;
}
virtual ObIndexInfoBlockHandler& get_flying_ilog_info_handler()
{
return fake_handler_;
}
void revert_flying_ilog_info_handler(ObIndexInfoBlockHandler* handler)
{
UNUSED(handler);
}
virtual int get_clog_info_handler(clog::file_id_t a, clog::ObCommitInfoBlockHandler& b)
{
UNUSED(a);
UNUSED(b);
return common::OB_SUCCESS;
}
virtual int get_ilog_info_handler(clog::file_id_t a, clog::ObIndexInfoBlockHandler& b, ObReadCost& cost)
{
UNUSED(a);
UNUSED(b);
UNUSED(cost);
return common::OB_SUCCESS;
}
virtual int get_remote_membership_status(const common::ObAddr& server, const common::ObPartitionKey& partition_key,
int64_t& timestamp, uint64_t& max_confirmed_log_id, bool& is_in_sync)
{
UNUSED(server);
UNUSED(partition_key);
UNUSED(timestamp);
UNUSED(max_confirmed_log_id);
UNUSED(is_in_sync);
return common::OB_SUCCESS;
}
virtual int get_follower_sync_info(const common::ObAddr& a, const common::ObPartitionKey& b, uint64_t& c, int64_t& d)
{
UNUSED(a);
UNUSED(b);
UNUSED(c);
UNUSED(d);
return common::OB_SUCCESS;
}
virtual int add_to_black_list(const common::ObAddr& server)
{
UNUSED(server);
return common::OB_SUCCESS;
}
virtual int remove_from_black_list(const common::ObAddr& server, bool& is_exec_remove)
{
UNUSED(server);
UNUSED(is_exec_remove);
return common::OB_SUCCESS;
}
virtual void try_limit_network(const ObLogEntry& log_entry)
{
UNUSED(log_entry);
}
virtual int64_t get_free_quota() const
{
return 0;
};
virtual bool is_using_new_log_env() const
{
return true;
}
ObILogDir* get_sys_ilog_dir()
{
return &si_dir_;
}
ObILogDir* get_user_ilog_dir()
{
return &ui_dir_;
}
ObILogDir* get_sys_clog_dir()
{
return &sc_dir_;
}
ObILogDir* get_user_clog_dir()
{
return &uc_dir_;
}
ObILogDir* get_clog_dir()
{
return NULL;
}
ObILogDir* get_ilog_dir()
{
return NULL;
}
int switch_clog_file()
{
return common::OB_SUCCESS;
}
bool is_disk_space_enough() const
{
return true;
}
/*
virtual int set_new_env_min_using_file_id(const bool is_sys,
const file_id_t ilog_file_id,
const file_id_t clog_file_id)
*/
int get_corresponding_clog_file_id(const file_id_t ilog_file_id, file_id_t& clog_file_id)
{
UNUSED(ilog_file_id);
UNUSED(clog_file_id);
return common::OB_SUCCESS;
}
virtual int broadcast_info(const common::ObMemberList& mem_list, const common::ObPartitionKey& key,
const common::ObReplicaType& replica_type, const uint64_t max_confirmed_log_id)
{
UNUSED(mem_list);
UNUSED(key);
UNUSED(replica_type);
UNUSED(max_confirmed_log_id);
return 0;
}
void set_all_scan_finish()
{}
virtual int set_env_min_using_file_id(const file_id_t ilog_file_id, const file_id_t clog_file_id)
{
UNUSED(ilog_file_id);
UNUSED(clog_file_id);
return common::OB_SUCCESS;
}
virtual int set_new_env_min_using_file_id(const file_id_t ilog_file_id, const file_id_t clog_file_id)
{
UNUSED(ilog_file_id);
UNUSED(clog_file_id);
return common::OB_SUCCESS;
}
virtual int submit_batch_log(const common::ObMemberList& member_list, const transaction::ObTransID& trans_id,
const common::ObPartitionArray& partition_array, const ObLogInfoArray& log_info_array)
{
UNUSED(member_list);
UNUSED(trans_id);
UNUSED(partition_array);
UNUSED(log_info_array);
return common::OB_SUCCESS;
}
virtual int submit_batch_ack(
const common::ObAddr& leader, const transaction::ObTransID& trans_id, const ObBatchAckArray& batch_ack_array)
{
UNUSED(leader);
UNUSED(trans_id);
UNUSED(batch_ack_array);
return common::OB_SUCCESS;
}
virtual int query_remote_log(const common::ObAddr& server, const common::ObPartitionKey& partition_key,
const uint64_t log_id, transaction::ObTransID& trans_id, int64_t& submit_timestamp)
{
UNUSED(server);
UNUSED(partition_key);
UNUSED(log_id);
UNUSED(trans_id);
UNUSED(submit_timestamp);
return common::OB_SUCCESS;
}
virtual void record_flush_cb_histogram(const int64_t cost_ts)
{
UNUSED(cost_ts);
}
virtual void record_batch_flush_cb_histogram(const int64_t cost_ts)
{
UNUSED(cost_ts);
}
virtual int get_cursor_batch(const common::ObPartitionKey& pkey, const uint64_t query_log_id,
ObGetCursorResult& result, ObIlogStorageQueryCost& csr_cost, const uint64_t retry_limit,
const unsigned sleep_time_on_retry_us)
{
int ret = OB_SUCCESS;
UNUSED(pkey);
UNUSED(query_log_id);
UNUSED(result);
UNUSED(csr_cost);
UNUSED(retry_limit);
UNUSED(sleep_time_on_retry_us);
return ret;
}
virtual int get_cursor_batch(const common::ObPartitionKey& pkey, const uint64_t query_log_id,
ObLogCursorExt& log_cursor, ObGetCursorResult& result, uint64_t& cursor_start_log_id)
{
int ret = OB_SUCCESS;
UNUSED(pkey);
UNUSED(query_log_id);
UNUSED(log_cursor);
UNUSED(result);
UNUSED(cursor_start_log_id);
return ret;
}
virtual int get_cursor(const common::ObPartitionKey& pkey, const uint64_t query_log_id, ObLogCursorExt& log_cursor)
{
int ret = OB_SUCCESS;
UNUSED(pkey);
UNUSED(query_log_id);
UNUSED(log_cursor);
return ret;
}
virtual int query_max_ilog_id(const common::ObPartitionKey& pkey, uint64_t& ret_max_ilog_id)
{
int ret = OB_SUCCESS;
UNUSED(pkey);
UNUSED(ret_max_ilog_id);
return ret;
}
virtual int query_ilog_file_id(
const common::ObPartitionKey& pkey, const uint64_t query_log_id, file_id_t& ilog_file_id)
{
int ret = OB_SUCCESS;
UNUSED(pkey);
UNUSED(query_log_id);
UNUSED(ilog_file_id);
return ret;
}
virtual int guarantee_extend_min()
{
int ret = OB_SUCCESS;
return ret;
}
virtual int locate_by_timestamp(const common::ObPartitionKey& pkey, const int64_t start_ts, uint64_t& target_log_id,
int64_t& target_log_timestamp)
{
int ret = OB_SUCCESS;
UNUSED(pkey);
UNUSED(start_ts);
UNUSED(target_log_id);
UNUSED(target_log_timestamp);
return ret;
}
private:
ObIndexInfoBlockHandler fake_handler_;
ObLogDir si_dir_;
ObLogDir ui_dir_;
ObLogDir sc_dir_;
ObLogDir uc_dir_;
};
/*
class ObILogNetTask
{
public:
~ObILogNetTask() {}
char *get_data_buffer() const{return NULL;}
common::ObProposalID get_proposal_id() const{
return 0;}
int64_t get_data_len() const{return 0;}
};
class ObLogNetTask : public ObILogNetTask
{
public:
ObLogNetTask(common::ObProposalID proposal_id, char *buff,
int64_t data_len) : proposal_id_(proposal_id),
buff_(buff), data_len_(data_len) {};
char *get_data_buffer() const {return buff_;}
common::ObProposalID get_proposal_id() const {return proposal_id_;}
int64_t get_data_len() const {return data_len_;}
private:
common::ObProposalID proposal_id_;
char *buff_;
int64_t data_len_;
};
*/
} // end namespace clog
} // end namespace oceanbase
#endif // OCEANBASE_UNITTEST_CLOG_MOCK_CONTAINER_MOCK_OB_LOG_ENGINE_H_

View File

@ -0,0 +1,297 @@
/**
* 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_UNITTEST_CLOG_MOCK_LOG_MEMBERSHIPMGR
#define OCEANBASE_UNITTEST_CLOG_MOCK_LOG_MEMBERSHIPMGR
#include "clog/ob_log_membership_mgr_V2.h"
#include "clog/ob_i_submit_log_cb.h"
#include "clog/ob_log_define.h"
#include "clog/ob_log_type.h"
#include "common/ob_member_list.h"
#include "common/ob_partition_key.h"
namespace oceanbase {
namespace election {
class ObIElectionMgr;
}
namespace clog {
class ObILogSWForMS;
class ObILogStateMgrForMS;
class ObILogCallbackEngine;
class ObLogEntry;
class ObLogEntryHeader;
class MockObLogMembershipMgr : public ObILogMembershipMgr, public ObISubmitLogCb {
public:
MockObLogMembershipMgr()
{}
virtual ~MockObLogMembershipMgr()
{}
public:
int init(const common::ObMemberList& member_list, const int64_t membership_timestamp,
const uint64_t membership_log_id, const int64_t replica_num, const common::ObAddr& self,
const common::ObPartitionKey& partition_key, ObILogSWForMS* sw, ObILogStateMgrForMS* state_mgr,
ObILogCallbackEngine* cb_engine, election::ObIElectionMgr* election_mgr)
{
UNUSED(member_list);
UNUSED(membership_timestamp);
UNUSED(membership_log_id);
UNUSED(replica_num);
UNUSED(self);
UNUSED(partition_key);
UNUSED(sw);
UNUSED(state_mgr);
UNUSED(cb_engine);
UNUSED(election_mgr);
return OB_SUCCESS;
}
bool is_state_changed()
{
return true;
}
int switch_state()
{
return OB_SUCCESS;
}
int receive_recovery_log(
const ObLogEntry& log_entry, const bool is_confirmed, const int64_t accum_checksum, const bool is_batch_committed)
{
UNUSED(log_entry);
UNUSED(is_confirmed);
UNUSED(accum_checksum);
UNUSED(is_batch_committed);
return OB_SUCCESS;
}
int add_member(const common::ObMember& member, const int64_t quorum, obrpc::ObMCLogInfo& log_info)
{
UNUSED(member);
UNUSED(quorum);
UNUSED(log_info);
return OB_SUCCESS;
}
int remove_member(const common::ObMember& member, const int64_t quorum, obrpc::ObMCLogInfo& log_info)
{
UNUSED(member);
UNUSED(quorum);
UNUSED(log_info);
return OB_SUCCESS;
}
int64_t get_replica_num() const
{
return 3;
}
common::ObReplicaType get_replica_type() const
{
return REPLICA_TYPE_MAX;
}
common::ObReplicaProperty get_replica_property() const
{
return ObReplicaProperty();
}
const common::ObMemberList& get_curr_member_list() const
{
return member_list_;
}
virtual int get_lower_level_replica_list(common::ObChildReplicaList& list) const
{
UNUSED(list);
return 0;
}
virtual int check_paxos_children() const
{
return 0;
}
virtual void reset_children_list()
{
return;
}
virtual int set_children_list_to_curr_member_list()
{
return 0;
}
virtual int try_add_member_to_children_list(
const ObMember& member, const common::ObRegion& region, const common::ObReplicaType replica_type)
{
UNUSED(member);
UNUSED(region);
UNUSED(replica_type);
return 0;
}
virtual int try_replace_add_to_children_list(const ObMember& member, const common::ObRegion& region,
const common::ObReplicaType replica_type, const int64_t next_ilog_ts)
{
UNUSED(member);
UNUSED(region);
UNUSED(replica_type);
UNUSED(next_ilog_ts);
return 0;
}
virtual int try_remove_server_from_children_list(const common::ObAddr& server)
{
UNUSED(server);
return 0;
}
virtual bool is_children_list_full()
{
return true;
}
virtual int receive_log(
const ObLogEntry& log_entry, const common::ObAddr& server, const int64_t cluster_id, const ReceiveLogType rl_type)
{
UNUSED(log_entry);
UNUSED(server);
UNUSED(cluster_id);
UNUSED(rl_type);
return 0;
}
virtual int append_disk_log(const ObLogEntry& log_entry, const ObLogCursor& log_cursor, const int64_t accum_checksum,
const bool batch_committed)
{
UNUSED(log_entry);
UNUSED(log_cursor);
UNUSED(accum_checksum);
UNUSED(batch_committed);
return 0;
}
virtual int64_t get_timestamp() const
{
return 0;
}
virtual uint64_t get_log_id() const
{
return 0;
}
virtual void reset_status()
{
return;
}
virtual int write_start_membership(const ObLogType& log_type)
{
UNUSED(log_type);
return 0;
}
virtual bool is_state_init() const
{
return true;
}
virtual void reset_follower_pending_entry()
{
return;
}
virtual void submit_success_cb_task(const ObLogType& log_type, const uint64_t log_id, const char* log_buf,
const int64_t log_buf_len, const common::ObProposalID& proposal_id)
{
UNUSED(log_type);
UNUSED(log_id);
UNUSED(log_buf);
UNUSED(log_buf_len);
UNUSED(proposal_id);
return;
}
virtual void reconfirm_update_status(const ObLogEntry& log_entry)
{
UNUSED(log_entry);
return;
}
virtual int force_set_as_single_replica()
{
return 0;
}
virtual int force_set_replica_num(const int64_t replica_num)
{
UNUSED(replica_num);
return 0;
}
virtual int set_replica_type(const enum ObReplicaType replica_type)
{
UNUSED(replica_type);
return 0;
}
virtual int change_member_list_to_self(const int64_t new_membership_version)
{
UNUSED(new_membership_version);
return OB_SUCCESS;
}
virtual int on_success(const common::ObPartitionKey& partition_key, const clog::ObLogType log_type,
const uint64_t log_id, const int64_t version, const bool batch_committed, const bool batch_last_succeed)
{
UNUSED(partition_key);
UNUSED(log_type);
UNUSED(log_id);
UNUSED(version);
UNUSED(batch_committed);
UNUSED(batch_last_succeed);
return 0;
}
virtual int inc_replica_num(const int64_t replica_num)
{
UNUSED(replica_num);
return OB_NOT_SUPPORTED;
}
virtual int dec_replica_num(const int64_t replica_num)
{
UNUSED(replica_num);
return OB_NOT_SUPPORTED;
}
virtual int change_quorum(const common::ObMemberList& curr_member_list, const int64_t curr_quorum,
const int64_t new_quorum, obrpc::ObMCLogInfo& log_info)
{
UNUSED(curr_member_list);
UNUSED(curr_quorum);
UNUSED(new_quorum);
UNUSED(log_info);
return 0;
}
virtual common::ObProposalID get_ms_proposal_id() const
{
return ms_proposal_id_;
}
virtual int update_ms_proposal_id(const common::ObProposalID& ms_proposal_id)
{
UNUSED(ms_proposal_id);
return OB_SUCCESS;
}
virtual bool has_elect_mlist() const
{
return false;
}
virtual int get_curr_ms_log_body(ObLogEntryHeader& log_entry_header, char* buffer, const int64_t buf_len) const
{
UNUSED(log_entry_header);
UNUSED(buffer);
UNUSED(buf_len);
return OB_SUCCESS;
}
virtual int reset_renew_ms_log_task()
{
return OB_SUCCESS;
}
virtual bool is_renew_ms_log_majority_success() const
{
return true;
}
virtual int check_renew_ms_log_sync_state() const
{
return OB_SUCCESS;
}
private:
common::ObMemberList member_list_;
common::ObProposalID ms_proposal_id_;
};
} // namespace clog
} // namespace oceanbase
#endif // OCEANBASE_UNITTEST_CLOG_MOCK_LOG_MEMBERSHIPMGR

View File

@ -0,0 +1,153 @@
/**
* 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_UNITTEST_CLOG_CLOG_MOCK_UTILS_MOCK_LOG_RECONFIRM_H_
#define _OCEANBASE_UNITTEST_CLOG_CLOG_MOCK_UTILS_MOCK_LOG_RECONFIRM_H_
#include "lib/lock/ob_spin_lock.h"
#include "clog/ob_log_define.h"
#include "clog/ob_log_sliding_window.h"
#include "clog/ob_log_entry.h"
#include "clog/ob_log_entry_header.h"
#include "clog/ob_log_task.h"
#include "clog/ob_log_checksum_V2.h"
#include "clog/ob_log_reconfirm.h"
namespace oceanbase {
using namespace common;
namespace clog {
class ObILogMembershipMgr;
class MockObLogReconfirm : public ObLogReconfirm {
public:
MockObLogReconfirm()
{}
virtual ~MockObLogReconfirm()
{}
int init(ObILogSWForReconfirm* sw, ObILogStateMgrForReconfirm* state_mgr, ObILogMembershipMgr* mm,
ObILogEngine* log_engine, ObILogAllocator* alloc_mgr, const common::ObPartitionKey& partition_key,
const common::ObAddr self)
{
UNUSED(sw);
UNUSED(log_engine);
UNUSED(partition_key);
UNUSED(state_mgr);
UNUSED(mm);
UNUSED(alloc_mgr);
UNUSED(self);
return OB_SUCCESS;
}
int reconfirm()
{
return OB_SUCCESS;
}
int receive_log(const ObLogEntry& log_entry, const int64_t server_id)
{
UNUSED(log_entry);
UNUSED(server_id);
return OB_SUCCESS;
}
int receive_max_log_id(const int64_t server_id, const uint64_t log_id)
{
UNUSED(server_id);
UNUSED(log_id);
return OB_SUCCESS;
}
void clear()
{}
bool need_start_up()
{
return true;
}
enum State {
INITED = 0,
FLUSHING_PREPARE_LOG = 1,
FETCH_MAX_LSN = 2,
RECONFIRMING = 3,
START_WORKING = 4,
FINISHED = 5,
};
static const int64_t MAJORITY_TAG_BIT = 31;
static const int64_t CONFIRMED_TAG_BIT = 30;
static const int64_t SUBMITED_TAG_BIT = 29;
static const int64_t LOG_EXIST_TAG_BIT = 28;
private:
int init_reconfirm_()
{
return OB_SUCCESS;
}
bool is_new_proposal_id_flushed_()
{
return true;
}
int fetch_max_log_id_()
{
return OB_SUCCESS;
}
int try_set_majority_ack_tag_of_max_log_id_()
{
return OB_SUCCESS;
}
int get_start_id_and_leader_ts_()
{
return OB_SUCCESS;
}
int prepare_log_map_()
{
return OB_SUCCESS;
}
int reconfirm_log_()
{
return OB_SUCCESS;
}
int try_fetch_log_()
{
return OB_SUCCESS;
}
int confirm_log_()
{
return OB_SUCCESS;
}
int retry_confirm_log_()
{
return OB_SUCCESS;
}
int try_filter_invalid_log_()
{
return OB_SUCCESS;
}
bool is_first_time_()
{
return true;
}
bool need_retry_reconfirm_()
{
return true;
}
const char* get_state_str_(const State state) const
{
UNUSED(state);
return NULL;
}
void generate_nop_log_(const uint64_t log_id, const int64_t idx)
{
UNUSED(log_id);
UNUSED(idx);
}
};
} // namespace clog
} // namespace oceanbase
#endif // _OCEANBASE_UNITTEST_CLOG_CLOG_MOCK_UTILS_MOCK_LOG_RECONFIRM_H_

View File

@ -0,0 +1,58 @@
/**
* 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_UNITTEST_CLOG_MOCK_OB_LOG_REPLAY_ENGINE_WRAPPER_H_
#define OOCEANBASE_UNITTEST_CLOG_MOCK_OB_LOG_REPLAY_ENGINE_WRAPPER_H_
#include "clog/ob_log_replay_engine_wrapper.h"
#include "storage/replayengine/ob_i_log_replay_engine.h"
namespace oceanbase {
namespace clog {
class MockObLogReplayEngineWrapper : public ObLogReplayEngineWrapper {
public:
MockObLogReplayEngineWrapper()
{}
virtual ~MockObLogReplayEngineWrapper()
{}
public:
int init(replayengine::ObILogReplayEngine* log_replay_engine)
{
UNUSED(log_replay_engine);
return OB_SUCCESS;
}
public:
int submit_replay_task(const common::ObPartitionKey& partition_key, const ObLogEntry& log_entry)
{
UNUSED(partition_key);
UNUSED(log_entry);
return OB_SUCCESS;
}
bool is_replay_finished(const common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
int is_tenant_out_of_memory(const common::ObPartitionKey& partition_key, bool& is_out_of_mem)
{
UNUSED(partition_key);
UNUSED(is_out_of_mem);
return OB_SUCCESS;
}
};
} // namespace clog
} // namespace oceanbase
#endif // OCEANBASE_UNITTEST_CLOG_MOCK_OB_LOG_REPLAY_ENGINE_WRAPPER_H_

View File

@ -0,0 +1,52 @@
/**
* 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_UNITTEST_CLOG_MOCK_OB_LOG_SLIDING_WINDOW_H_
#define _OCEANBASE_UNITTEST_CLOG_MOCK_OB_LOG_SLIDING_WINDOW_H_
#include "lib/atomic/atomic128.h"
#include "clog/ob_log_define.h"
#include "clog/ob_log_state_mgr.h"
#include "clog/ob_log_entry.h"
#include "clog/ob_log_entry_header.h"
#include "clog/ob_log_task.h"
#include "clog/ob_log_flush_task.h"
#include "clog/ob_i_disk_log_buffer.h"
#include "clog/ob_log_checksum_V2.h"
#include "clog/ob_fetch_log_engine.h"
namespace oceanbase {
namespace clog {
class ObILogReplayEngineWrapper;
class ObIOutOfBandLogHandler;
class ObILogStateMgrForSW;
class MockObSlidingCallBack : public ObILogTaskCallBack {
public:
MockObSlidingCallBack()
{}
virtual ~MockObSlidingCallBack()
{}
int sliding_cb(const int64_t sn, const ObILogExtRingBufferData* data)
{
UNUSED(sn);
UNUSED(data);
return OB_SUCCESS;
}
void inc_task_num()
{}
void dec_task_num()
{}
};
} // namespace clog
} // namespace oceanbase
#endif //_OCEANBASE_UNITTEST_CLOG_MOCK_OB_LOG_SLIDING_WINDOW_H_

View File

@ -0,0 +1,235 @@
/**
* 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_CLOG_MOCK_OB_LOG_STATE_MGR_H_
#define OCEANBASE_CLOG_MOCK_OB_LOG_STATE_MGR_H_
#include "clog/ob_log_state_mgr.h"
namespace oceanbase {
namespace election {
class ObIElectionMgr;
}
namespace share {
class ObIPSCb;
}
namespace clog {
class ObILogSWForStateMgr;
class ObILogReconfirm;
class ObILogEngine;
class ObILogMembershipMgr;
class ObILogReplayEngineWrapper;
class ObILogCallbackEngine;
class ObILogAllocator;
class MockObLogStateMgr : public ObLogStateMgr {
public:
MockObLogStateMgr()
{}
~MockObLogStateMgr()
{
destroy();
}
public:
int init(ObILogSWForStateMgr* sw, ObILogReconfirm* reconfirm, ObILogEngine* log_engine, ObILogMembershipMgr* mm,
election::ObIElectionMgr* election_mgr, ObILogReplayEngineWrapper* replay_engine, share::ObIPSCb* ps_cb,
ObILogAllocator* alloc_mgr, const common::ObAddr& self, const common::ObProposalID& proposal_id,
const common::ObVersion& freeze_version, const common::ObPartitionKey& partition_key)
{
UNUSED(sw);
UNUSED(reconfirm);
UNUSED(log_engine);
UNUSED(mm);
UNUSED(election_mgr);
UNUSED(replay_engine);
UNUSED(ps_cb);
UNUSED(alloc_mgr);
UNUSED(self);
UNUSED(proposal_id);
UNUSED(freeze_version);
UNUSED(partition_key);
return OB_SUCCESS;
}
bool can_submit_log() const
{
return true;
}
bool can_append_disk_log() const
{
return true;
}
bool can_majority_cb(const common::ObProposalID& proposal_id) const
{
UNUSED(proposal_id);
return true;
}
bool can_receive_log_ack(const common::ObProposalID& proposal_id) const
{
UNUSED(proposal_id);
return true;
}
bool can_send_log_ack(const common::ObProposalID& proposal_id) const
{
UNUSED(proposal_id);
return true;
}
bool can_receive_max_log_id(const common::ObProposalID& proposal_id) const
{
UNUSED(proposal_id);
return true;
}
bool can_get_log() const
{
return true;
}
bool can_get_log(const common::ObProposalID& proposal_id) const
{
UNUSED(proposal_id);
return true;
}
bool can_receive_log(const common::ObProposalID& proposal_id, const common::ObProposalID& proposal_id_in_log) const
{
UNUSED(proposal_id);
UNUSED(proposal_id_in_log);
return true;
}
bool can_change_member() const
{
return true;
}
bool can_get_base_storage_info() const
{
return true;
}
bool can_change_leader() const
{
return true;
}
bool can_get_leader_curr_member_list() const
{
return true;
}
bool can_handle_prepare_rqst(const common::ObProposalID& proposal_id) const
{
UNUSED(proposal_id);
return true;
}
bool can_slide_sw() const
{
return true;
}
bool can_receive_confirmed_info() const
{
return true;
}
bool need_quicker_polling() const
{
return true;
}
common::ObProposalID get_proposal_id() const
{
common::ObProposalID proposal_id;
return proposal_id;
}
int switch_state()
{
return OB_SUCCESS;
}
bool is_state_changed()
{
return true;
}
int set_scan_disk_log_finished()
{
return OB_SUCCESS;
}
ObLogState get_state() const
{
ObLogState state = UNKNOWN;
return state;
}
common::ObRole get_role() const
{
return INVALID_ROLE;
}
common::ObAddr get_leader() const
{
common::ObAddr leader;
return leader;
}
common::ObVersion get_freeze_version() const
{
common::ObVersion freeze_version;
return freeze_version;
}
int update_freeze_version(const common::ObVersion& freeze_version)
{
UNUSED(freeze_version);
return OB_SUCCESS;
}
int handle_prepare_rqst(const common::ObProposalID& proposal_id, const common::ObAddr& new_leader)
{
UNUSED(proposal_id);
UNUSED(new_leader);
return OB_SUCCESS;
}
int update_proposal_id(const common::ObProposalID& proposal_id)
{
UNUSED(proposal_id);
return OB_SUCCESS;
}
int stop_election()
{
return OB_SUCCESS;
}
int set_election_leader(const common::ObAddr& leader, const int64_t lease_start)
{
UNUSED(leader);
UNUSED(lease_start);
return OB_SUCCESS;
}
void set_removed()
{}
void set_temporary_replica()
{}
void reset_temporary_replica()
{}
bool is_temporary_replica() const
{
return true;
}
bool is_changing_leader() const
{
return true;
}
int change_leader_async(const common::ObPartitionKey& partition_key, const ObAddr& leader)
{
UNUSED(partition_key);
UNUSED(leader);
return OB_SUCCESS;
}
bool is_inited() const
{
return true;
}
void destroy()
{}
DISALLOW_COPY_AND_ASSIGN(MockObLogStateMgr);
};
} // namespace clog
} // namespace oceanbase
#endif // OCEANBASE_CLOG_OB_LOG_STATE_MGR_V2_H_

View File

@ -0,0 +1,110 @@
/**
* 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_UNUTTEST_CLOG_MOCK_OB_PS_CB_H_
#define OCEANBASE_UNUTTEST_CLOG_MOCK_OB_PS_CB_H_
#include "share/ob_i_ps_cb.h"
#include "common/ob_partition_key.h"
#include "common/ob_member_list.h"
namespace oceanbase {
namespace common {
class MockObPSCb : public share::ObIPSCb {
public:
MockObPSCb()
{}
virtual ~MockObPSCb()
{}
public:
int64_t get_min_using_file_id() const
{
return 0;
}
int on_leader_revoke(const common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
int on_leader_takeover(const common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
int on_member_change_success(const common::ObPartitionKey& partition_key, const int64_t mc_timestamp,
const common::ObMemberList& prev_member_list, const common::ObMemberList& curr_member_list)
{
UNUSED(partition_key);
UNUSED(mc_timestamp);
UNUSED(prev_member_list);
UNUSED(curr_member_list);
return OB_SUCCESS;
}
int on_leader_active(const oceanbase::common::ObPartitionKey& partition_key)
{
UNUSED(partition_key);
return OB_SUCCESS;
}
bool is_take_over_done(const oceanbase::common::ObPartitionKey& pkey) const
{
UNUSED(pkey);
return true;
}
bool is_revoke_done(const oceanbase::common::ObPartitionKey& pkey) const
{
UNUSED(pkey);
return true;
}
virtual bool is_tenant_active(const common::ObPartitionKey& pkey)
{
UNUSED(pkey);
return true;
}
virtual int get_leader_from_loc_cache(const common::ObPartitionKey& pkey, ObAddr& leader, const bool is_need_renew)
{
UNUSED(pkey);
UNUSED(leader);
UNUSED(is_need_renew);
return common::OB_SUCCESS;
}
virtual int handle_log_missing(const common::ObPartitionKey& pkey, const common::ObAddr& server)
{
UNUSED(pkey);
UNUSED(server);
return common::OB_SUCCESS;
}
virtual int get_server_locality_array(
common::ObIArray<share::ObServerLocality>& server_locality_array, bool& has_readonly_zone) const
{
UNUSED(server_locality_array);
UNUSED(has_readonly_zone);
return common::OB_SUCCESS;
}
virtual int check_partition_index_available(const common::ObPartitionKey& pkey, bool& available)
{
UNUSED(pkey);
UNUSED(available);
return common::OB_SUCCESS;
}
virtual int get_last_ssstore_version(
const oceanbase::common::ObPartitionKey& pkey, oceanbase::common::ObVersion& version)
{
UNUSED(pkey);
UNUSED(version);
return common::OB_SUCCESS;
}
};
} // namespace common
} // namespace oceanbase
#endif // OCEANBASE_UNUTTEST_CLOG_MOCK_OB_PS_CB_H_

View File

@ -0,0 +1,49 @@
/**
* 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_CLOG_MOCK_OB_SUBMIT_LOG_CB_H_
#define OCEANBASE_CLOG_MOCK_OB_SUBMIT_LOG_CB_H_
#include "clog/ob_i_submit_log_cb.h"
namespace oceanbase {
namespace clog {
class MockObSubmitLogCb : public ObISubmitLogCb {
public:
MockObSubmitLogCb()
{}
virtual ~MockObSubmitLogCb()
{}
public:
int on_success(const common::ObPartitionKey& partition_key, const clog::ObLogType log_type, const uint64_t log_id,
const int64_t version, const bool batch_committed, const bool batch_last_succeed)
{
UNUSED(partition_key);
UNUSED(log_type);
UNUSED(log_id);
UNUSED(version);
UNUSED(batch_committed);
UNUSED(batch_last_succeed);
return common::OB_SUCCESS;
}
int on_finished(const common::ObPartitionKey& partition_key, const uint64_t log_id)
{
int ret = common::OB_SUCCESS;
UNUSED(partition_key);
UNUSED(log_id);
return ret;
}
};
} // namespace clog
} // namespace oceanbase
#endif // OCEANBASE_CLOG_MOCK_OB_SUBMIT_LOG_CB_H_