add _rebuild_replica_log_lag_threshold for test

This commit is contained in:
yyy-hust
2023-05-25 22:41:42 +00:00
committed by ob-robot
parent ff9b54bb1d
commit 16288e642b
11 changed files with 60 additions and 9 deletions

View File

@ -285,7 +285,11 @@ int ObArchiveSender::do_consume_send_task_()
// As task issued flag is marked, no matter task is handled succ or fail // As task issued flag is marked, no matter task is handled succ or fail
// the flag should be dealed. // the flag should be dealed.
if (OB_FAIL(get_send_task_(task, task_exist))) { if (OB_FAIL(get_send_task_(task, task_exist))) {
ARCHIVE_LOG(WARN, "get send task failed", K(ret)); if (OB_ENTRY_NOT_EXIST != ret) {
ARCHIVE_LOG(WARN, "get send task failed", K(ret));
} else {
ARCHIVE_LOG(TRACE, "get send task failed", K(ret));
}
} else if (! task_exist) { } else if (! task_exist) {
} else if (FALSE_IT(handle(*task, consume_status))) { } else if (FALSE_IT(handle(*task, consume_status))) {
} else { } else {

View File

@ -520,10 +520,11 @@ int ObLogService::update_palf_options_except_disk_usage_limit_size()
palf_opts.disk_options_.log_disk_throttling_percentage_ = tenant_config->log_disk_throttling_percentage; palf_opts.disk_options_.log_disk_throttling_percentage_ = tenant_config->log_disk_throttling_percentage;
palf_opts.compress_options_.enable_transport_compress_ = tenant_config->log_transport_compress_all; palf_opts.compress_options_.enable_transport_compress_ = tenant_config->log_transport_compress_all;
palf_opts.compress_options_.transport_compress_func_ = compressor_type; palf_opts.compress_options_.transport_compress_func_ = compressor_type;
palf_opts.rebuild_replica_log_lag_threshold_ = tenant_config->_rebuild_replica_log_lag_threshold;
if (OB_FAIL(palf_env_->update_options(palf_opts))) { if (OB_FAIL(palf_env_->update_options(palf_opts))) {
CLOG_LOG(WARN, "palf update_options failed", K(MTL_ID()), K(ret)); CLOG_LOG(WARN, "palf update_options failed", K(MTL_ID()), K(ret), K(palf_opts));
} else { } else {
CLOG_LOG(INFO, "palf update_options success", K(MTL_ID()), K(ret)); CLOG_LOG(INFO, "palf update_options success", K(MTL_ID()), K(ret), K(palf_opts));
} }
} }
} }

View File

@ -230,12 +230,20 @@ private:
if (true == matched_type) { if (true == matched_type) {
if (OB_FAIL(curr_entry_.deserialize(buf_, curr_read_buf_end_pos_, pos))) { if (OB_FAIL(curr_entry_.deserialize(buf_, curr_read_buf_end_pos_, pos))) {
} else if (OB_FAIL(handle_each_log_group_entry_(curr_entry_, replayable_point_scn, info))) { } else if (OB_FAIL(handle_each_log_group_entry_(curr_entry_, replayable_point_scn, info))) {
PALF_LOG(WARN, "handle_each_log_group_entry_ failed", KPC(this), K(info), K(replayable_point_scn)); if (OB_ITER_END != ret) {
PALF_LOG(WARN, "handle_each_log_group_entry_ failed", KPC(this), K(info), K(replayable_point_scn));
} else {
PALF_LOG(TRACE, "handle_each_log_group_entry_ failed", KPC(this), K(info), K(replayable_point_scn));
}
} }
} else if (OB_FAIL(actual_entry.deserialize(buf_, curr_read_buf_end_pos_, pos))) { } else if (OB_FAIL(actual_entry.deserialize(buf_, curr_read_buf_end_pos_, pos))) {
PALF_LOG(TRACE, "deserialize entry failed", K(ret), KPC(this)); PALF_LOG(TRACE, "deserialize entry failed", K(ret), KPC(this));
} else if (OB_FAIL(handle_each_log_group_entry_(actual_entry, replayable_point_scn, info))) { } else if (OB_FAIL(handle_each_log_group_entry_(actual_entry, replayable_point_scn, info))) {
PALF_LOG(WARN, "handle_each_log_group_entry_ failed", KPC(this), K(actual_entry), K(info), K(replayable_point_scn)); if (OB_ITER_END != ret) {
PALF_LOG(WARN, "handle_each_log_group_entry_ failed", KPC(this), K(actual_entry), K(info), K(replayable_point_scn));
} else {
PALF_LOG(TRACE, "handle_each_log_group_entry_ failed", KPC(this), K(actual_entry), K(info), K(replayable_point_scn));
}
} else { } else {
ret = OB_EAGAIN; ret = OB_EAGAIN;
advance_read_lsn_(actual_entry.get_payload_offset()); advance_read_lsn_(actual_entry.get_payload_offset());

View File

@ -171,6 +171,7 @@ PalfEnvImpl::PalfEnvImpl() : palf_meta_lock_(common::ObLatchIds::PALF_ENV_LOCK),
self_(), self_(),
palf_handle_impl_map_(64), // 指定min_size=64 palf_handle_impl_map_(64), // 指定min_size=64
last_palf_epoch_(0), last_palf_epoch_(0),
rebuild_replica_log_lag_threshold_(0),
diskspace_enough_(true), diskspace_enough_(true),
tenant_id_(0), tenant_id_(0),
is_inited_(false), is_inited_(false),
@ -333,6 +334,7 @@ void PalfEnvImpl::destroy()
log_dir_[0] = '\0'; log_dir_[0] = '\0';
tmp_log_dir_[0] = '\0'; tmp_log_dir_[0] = '\0';
disk_options_wrapper_.reset(); disk_options_wrapper_.reset();
rebuild_replica_log_lag_threshold_ = 0;
} }
// NB: not thread safe // NB: not thread safe
@ -851,6 +853,7 @@ int PalfEnvImpl::update_options(const PalfOptions &options)
} else if (OB_FAIL(log_rpc_.update_transport_compress_options(options.compress_options_))) { } else if (OB_FAIL(log_rpc_.update_transport_compress_options(options.compress_options_))) {
PALF_LOG(WARN, "update_transport_compress_options failed", K(ret), K(options)); PALF_LOG(WARN, "update_transport_compress_options failed", K(ret), K(options));
} else { } else {
rebuild_replica_log_lag_threshold_ = options.rebuild_replica_log_lag_threshold_;
PALF_LOG(INFO, "update_palf_options success", K(options)); PALF_LOG(INFO, "update_palf_options success", K(options));
} }
return ret; return ret;
@ -864,6 +867,7 @@ int PalfEnvImpl::get_options(PalfOptions &options)
} else { } else {
options.disk_options_ = disk_options_wrapper_.get_disk_opts_for_recycling_blocks(); options.disk_options_ = disk_options_wrapper_.get_disk_opts_for_recycling_blocks();
options.compress_options_ = log_rpc_.get_compress_opts(); options.compress_options_ = log_rpc_.get_compress_opts();
options.rebuild_replica_log_lag_threshold_ = rebuild_replica_log_lag_threshold_;
} }
return ret; return ret;
} }

View File

@ -174,6 +174,7 @@ public:
virtual int create_directory(const char *base_dir) = 0; virtual int create_directory(const char *base_dir) = 0;
virtual int remove_directory(const char *base_dir) = 0; virtual int remove_directory(const char *base_dir) = 0;
virtual bool check_disk_space_enough() = 0; virtual bool check_disk_space_enough() = 0;
virtual int64_t get_rebuild_replica_log_lag_threshold() const = 0;
virtual int get_io_start_time(int64_t &last_working_time) = 0; virtual int get_io_start_time(int64_t &last_working_time) = 0;
virtual int64_t get_tenant_id() = 0; virtual int64_t get_tenant_id() = 0;
// should be removed in version 4.2.0.0 // should be removed in version 4.2.0.0
@ -238,6 +239,8 @@ public:
int get_disk_usage(int64_t &used_size_byte, int64_t &total_usable_size_byte); int get_disk_usage(int64_t &used_size_byte, int64_t &total_usable_size_byte);
int update_options(const PalfOptions &options); int update_options(const PalfOptions &options);
int get_options(PalfOptions &options); int get_options(PalfOptions &options);
int64_t get_rebuild_replica_log_lag_threshold() const
{return rebuild_replica_log_lag_threshold_;}
int for_each(const common::ObFunction<int(const PalfHandle&)> &func); int for_each(const common::ObFunction<int(const PalfHandle&)> &func);
int for_each(const common::ObFunction<int(IPalfHandleImpl *ipalf_handle_impl)> &func) override final; int for_each(const common::ObFunction<int(IPalfHandleImpl *ipalf_handle_impl)> &func) override final;
common::ObILogAllocator* get_log_allocator() override final; common::ObILogAllocator* get_log_allocator() override final;
@ -349,6 +352,7 @@ private:
// last_palf_epoch_ is used to assign increasing epoch for each palf instance. // last_palf_epoch_ is used to assign increasing epoch for each palf instance.
int64_t last_palf_epoch_; int64_t last_palf_epoch_;
int64_t rebuild_replica_log_lag_threshold_;//for rebuild test
LogIOWorkerConfig log_io_worker_config_; LogIOWorkerConfig log_io_worker_config_;
bool diskspace_enough_; bool diskspace_enough_;

View File

@ -82,6 +82,7 @@ PalfHandleImpl::PalfHandleImpl()
last_check_sync_time_us_(OB_INVALID_TIMESTAMP), last_check_sync_time_us_(OB_INVALID_TIMESTAMP),
last_renew_loc_time_us_(OB_INVALID_TIMESTAMP), last_renew_loc_time_us_(OB_INVALID_TIMESTAMP),
last_print_in_sync_time_us_(OB_INVALID_TIMESTAMP), last_print_in_sync_time_us_(OB_INVALID_TIMESTAMP),
last_hook_fetch_log_time_us_(OB_INVALID_TIMESTAMP),
chaning_config_warn_time_(OB_INVALID_TIMESTAMP), chaning_config_warn_time_(OB_INVALID_TIMESTAMP),
cached_is_in_sync_(false), cached_is_in_sync_(false),
has_higher_prio_config_change_(false), has_higher_prio_config_change_(false),
@ -3122,6 +3123,8 @@ int PalfHandleImpl::fetch_log_from_storage_(const common::ObAddr &server,
PALF_LOG(INFO, "the LSN between leader and non paxos member is not same, do not fetch log", PALF_LOG(INFO, "the LSN between leader and non paxos member is not same, do not fetch log",
K_(palf_id), K(fetch_start_lsn), K(prev_log_info)); K_(palf_id), K(fetch_start_lsn), K(prev_log_info));
} }
} else if (check_need_hook_fetch_log_(fetch_type, fetch_start_lsn)) {
ret = OB_ERR_OUT_OF_LOWER_BOUND;
} else if (OB_FAIL(iterator.init(fetch_start_lsn, get_file_end_lsn, log_engine_.get_log_storage()))) { } else if (OB_FAIL(iterator.init(fetch_start_lsn, get_file_end_lsn, log_engine_.get_log_storage()))) {
PALF_LOG(WARN, "PalfGroupBufferIterator init failed", K(ret), K_(palf_id)); PALF_LOG(WARN, "PalfGroupBufferIterator init failed", K(ret), K_(palf_id));
} else { } else {
@ -4595,6 +4598,24 @@ void PalfHandleImpl::report_switch_acceptor_to_learner_(const common::ObMember &
replica_type_to_string(ObReplicaType::REPLICA_TYPE_FULL, replica_full_name_, sizeof(replica_full_name_)); replica_type_to_string(ObReplicaType::REPLICA_TYPE_FULL, replica_full_name_, sizeof(replica_full_name_));
plugins_.record_replica_type_change_event(palf_id_, config_version, replica_full_name_, replica_readonly_name_, EXTRA_INFOS); plugins_.record_replica_type_change_event(palf_id_, config_version, replica_full_name_, replica_readonly_name_, EXTRA_INFOS);
} }
bool PalfHandleImpl::check_need_hook_fetch_log_(const FetchLogType fetch_type, const LSN &start_lsn)
{
bool bool_ret = false;
const int64_t rebuild_replica_log_lag_threshold = palf_env_impl_->get_rebuild_replica_log_lag_threshold();
if (rebuild_replica_log_lag_threshold > 0 && (FETCH_LOG_FOLLOWER == fetch_type)) {
LSN max_lsn = get_max_lsn();
LSN base_lsn = get_base_lsn_used_for_block_gc();
bool_ret = (start_lsn < base_lsn) && ((max_lsn - start_lsn) > rebuild_replica_log_lag_threshold);
if (bool_ret && palf_reach_time_interval(1 * 1000 * 1000L, last_hook_fetch_log_time_us_)) {
PALF_LOG(INFO, "hook fetch_log because of rebuild_replica_log_lag_threshold", K(palf_id_),
K(rebuild_replica_log_lag_threshold), K(start_lsn), K(max_lsn), K(base_lsn));
}
}
return bool_ret;
}
PalfStat::PalfStat() PalfStat::PalfStat()
: self_(), : self_(),
palf_id_(INVALID_PALF_ID), palf_id_(INVALID_PALF_ID),

View File

@ -1095,6 +1095,7 @@ private:
void report_switch_learner_to_acceptor_(const common::ObMember &learner); void report_switch_learner_to_acceptor_(const common::ObMember &learner);
void report_switch_acceptor_to_learner_(const common::ObMember &acceptor); void report_switch_acceptor_to_learner_(const common::ObMember &acceptor);
// ======================= report event end ======================================= // ======================= report event end =======================================
bool check_need_hook_fetch_log_(const FetchLogType fetch_type, const LSN &start_lsn);
private: private:
class ElectionMsgSender : public election::ElectionMsgSender class ElectionMsgSender : public election::ElectionMsgSender
{ {
@ -1205,6 +1206,7 @@ private:
int64_t last_check_sync_time_us_; int64_t last_check_sync_time_us_;
int64_t last_renew_loc_time_us_; int64_t last_renew_loc_time_us_;
int64_t last_print_in_sync_time_us_; int64_t last_print_in_sync_time_us_;
int64_t last_hook_fetch_log_time_us_;
int64_t chaning_config_warn_time_; int64_t chaning_config_warn_time_;
bool cached_is_in_sync_; bool cached_is_in_sync_;
bool has_higher_prio_config_change_; bool has_higher_prio_config_change_;
@ -1212,5 +1214,4 @@ private:
}; };
} // end namespace palf } // end namespace palf
} // end namespace oceanbase } // end namespace oceanbase
#endif // OCEANBASE_LOGSERVICE_LOG_SERVICE_ #endif // OCEANBASE_LOGSERVICE_LOG_SERVICE_

View File

@ -24,11 +24,12 @@ void PalfOptions::reset()
{ {
disk_options_.reset(); disk_options_.reset();
compress_options_.reset(); compress_options_.reset();
rebuild_replica_log_lag_threshold_ = 0;
} }
bool PalfOptions::is_valid() const bool PalfOptions::is_valid() const
{ {
return disk_options_.is_valid() && compress_options_.is_valid(); return disk_options_.is_valid() && compress_options_.is_valid() && (rebuild_replica_log_lag_threshold_ >= 0);
} }
void PalfDiskOptions::reset() void PalfDiskOptions::reset()

View File

@ -162,16 +162,19 @@ public:
struct PalfOptions struct PalfOptions
{ {
PalfOptions() : disk_options_(), PalfOptions() : disk_options_(),
compress_options_() compress_options_(),
rebuild_replica_log_lag_threshold_(0)
{} {}
~PalfOptions() { reset(); } ~PalfOptions() { reset(); }
void reset(); void reset();
bool is_valid() const; bool is_valid() const;
TO_STRING_KV(K(disk_options_), TO_STRING_KV(K(disk_options_),
K(compress_options_)); K(compress_options_),
K(rebuild_replica_log_lag_threshold_));
public: public:
PalfDiskOptions disk_options_; PalfDiskOptions disk_options_;
PalfTransportCompressOptions compress_options_; PalfTransportCompressOptions compress_options_;
int64_t rebuild_replica_log_lag_threshold_;
}; };
struct PalfThrottleOptions struct PalfThrottleOptions

View File

@ -1482,3 +1482,6 @@ DEF_BOOL(_optimizer_group_by_placement, OB_TENANT_PARAMETER, "True",
DEF_TIME(_wait_interval_after_truncate, OB_CLUSTER_PARAMETER, "30s", "[0s,)", DEF_TIME(_wait_interval_after_truncate, OB_CLUSTER_PARAMETER, "30s", "[0s,)",
"time interval for waiting other servers to refresh schema after truncate", "time interval for waiting other servers to refresh schema after truncate",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_CAP(_rebuild_replica_log_lag_threshold, OB_TENANT_PARAMETER, "0M", "[0M,+∞]",
"size of clog files that a replica lag behind leader to trigger rebuild, 0 means never trigger rebuild on purpose. Range: [0, +∞)",
ObParameterAttr(Section::TRANS, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));

View File

@ -334,6 +334,7 @@ _px_max_message_pool_pct
_px_max_pipeline_depth _px_max_pipeline_depth
_px_message_compression _px_message_compression
_px_object_sampling _px_object_sampling
_rebuild_replica_log_lag_threshold
_recyclebin_object_purge_frequency _recyclebin_object_purge_frequency
_resource_limit_max_session_num _resource_limit_max_session_num
_resource_limit_spec _resource_limit_spec