[log throttling] add parameter log_disk_throttling_maximum_duration

This commit is contained in:
yyy-hust
2023-07-11 05:42:44 +00:00
committed by ob-robot
parent 7c63a546e7
commit d88e4bd320
18 changed files with 154 additions and 20 deletions

View File

@ -289,6 +289,7 @@ int ObSimpleLogServer::init_log_service_()
opts.disk_options_.log_disk_utilization_threshold_ = 80;
opts.disk_options_.log_disk_utilization_limit_threshold_ = 95;
opts.disk_options_.log_disk_throttling_percentage_ = 100;
opts.disk_options_.log_disk_throttling_maximum_duration_ = 2 * 3600 * 1000 * 1000L;
opts.disk_options_.log_writer_parallelism_ = 2;
std::string clog_dir = clog_dir_ + "/tenant_1";
allocator_ = OB_NEW(ObTenantMutilAllocator, "TestBase", node_id_);

View File

@ -277,9 +277,19 @@ TEST_F(TestObSimpleLogClusterLogThrottling, test_throttling_basic)
int64_t cur_has_batched_size = log_io_worker->batch_io_task_mgr_.has_batched_size_;
// no io reduce during writing throttling
ASSERT_EQ(cur_has_batched_size, prev_has_batched_size);
const double old_decay_factor = throttle->decay_factor_;
PALF_LOG(INFO, "[CASE 7] defactor is will change when log_disk_throttling_maximum_duration changes", K(max_lsn_1));
palf_env_impl.disk_options_wrapper_.disk_opts_for_stopping_writing_.log_disk_throttling_maximum_duration_ = 1800 * 1000 * 1000L;
usleep(LogWritingThrottle::UPDATE_INTERVAL_US);
PALF_LOG(INFO, "[CASE 7] need break from writing throttling while unrecyclable size fallbacks", K(max_lsn_1));
EXPECT_EQ(OB_SUCCESS, submit_log(leader, 20, id, 1024));
max_lsn_1 = leader.palf_handle_impl_->sw_.get_max_lsn();
wait_lsn_until_flushed(max_lsn_1, leader);
PALF_LOG(INFO, "YYY change when log_disk_throttling_maximum_duration changes", K(old_decay_factor), KPC(throttle));
ASSERT_EQ(true, throttle->decay_factor_ < old_decay_factor);
PALF_LOG(INFO, "[CASE 8] need break from writing throttling while unrecyclable size fallbacks", K(max_lsn_1));
EXPECT_EQ(OB_SUCCESS, submit_log(leader, 20, id, 1 * MB));
cur_ts = common::ObClockGenerator::getClock();

View File

@ -518,6 +518,7 @@ int ObLogService::update_palf_options_except_disk_usage_limit_size()
palf_opts.disk_options_.log_disk_utilization_threshold_ = tenant_config->log_disk_utilization_threshold;
palf_opts.disk_options_.log_disk_utilization_limit_threshold_ = tenant_config->log_disk_utilization_limit_threshold;
palf_opts.disk_options_.log_disk_throttling_percentage_ = tenant_config->log_disk_throttling_percentage;
palf_opts.disk_options_.log_disk_throttling_maximum_duration_ = tenant_config->log_disk_throttling_maximum_duration;
palf_opts.compress_options_.enable_transport_compress_ = tenant_config->log_transport_compress_all;
palf_opts.compress_options_.transport_compress_func_ = compressor_type;
palf_opts.rebuild_replica_log_lag_threshold_ = tenant_config->_rebuild_replica_log_lag_threshold;

View File

@ -37,8 +37,8 @@ public:
virtual int wait_append_sync(const share::ObLSID &ls_id);
private:
const int64_t MAX_SINGLE_REPLAY_WARNING_TIME_THRESOLD = 100 * 1000; //100ms
const int64_t MAX_SINGLE_REPLAY_ERROR_TIME_THRESOLD = 1000 * 1000; //1s 单条日志回放执行时间超过此值报error
const int64_t MAX_SINGLE_RETRY_WARNING_TIME_THRESOLD = 5 * 1000 * 1000; //1s 单条日志回放重试超过此值报error
const int64_t MAX_SINGLE_REPLAY_ERROR_TIME_THRESOLD = 2 * 1000 * 1000; //2s 单条日志回放执行时间超过此值报error
const int64_t MAX_SINGLE_RETRY_WARNING_TIME_THRESOLD = 5 * 1000 * 1000; //5s 单条日志回放重试超过此值报error
bool is_inited_;
storage::ObLSService *ls_service_;
};

View File

@ -146,6 +146,7 @@ int LogWritingThrottle::update_throtting_options_guarded_by_lock_(IPalfEnvImpl *
} else {
const bool need_throttling = new_throttling_options.need_throttling();
const int64_t new_available_size_after_limit = new_throttling_options.get_available_size_after_limit();
const int64_t new_maximum_duration = new_throttling_options.get_maximum_duration();
bool need_update_decay_factor = false;
bool need_start_throttling = false;
@ -154,16 +155,16 @@ int LogWritingThrottle::update_throtting_options_guarded_by_lock_(IPalfEnvImpl *
need_start_throttling = true;
need_update_decay_factor = true;
} else {
need_update_decay_factor = (throttling_options_.get_available_size_after_limit() != new_available_size_after_limit);
need_update_decay_factor = (throttling_options_.get_available_size_after_limit() != new_available_size_after_limit)
|| throttling_options_.get_maximum_duration() != new_maximum_duration;
}
if (need_update_decay_factor) {
if (OB_FAIL(ObThrottlingUtils::calc_decay_factor(new_available_size_after_limit, THROTTLING_DURATION_US,
if (OB_FAIL(ObThrottlingUtils::calc_decay_factor(new_available_size_after_limit, new_maximum_duration,
THROTTLING_CHUNK_SIZE, decay_factor_))) {
PALF_LOG(ERROR, "failed to calc_decay_factor", K(throttling_options_), "duration(s)",
THROTTLING_DURATION_US / (1000 * 1000), K(THROTTLING_CHUNK_SIZE));
PALF_LOG(ERROR, "failed to calc_decay_factor", K(throttling_options_), K(THROTTLING_CHUNK_SIZE));
} else {
PALF_LOG(INFO, "[LOG DISK THROTTLING] success to calc_decay_factor", K(decay_factor_), K(throttling_options_),
K(new_throttling_options), "duration(s)", THROTTLING_DURATION_US / (1000 * 1000L), K(THROTTLING_CHUNK_SIZE));
K(new_throttling_options), K(THROTTLING_CHUNK_SIZE));
}
}
@ -178,14 +179,12 @@ int LogWritingThrottle::update_throtting_options_guarded_by_lock_(IPalfEnvImpl *
throttling_options_ = new_throttling_options;
if (need_start_throttling) {
stat_.start_throttling();
PALF_LOG(INFO, "[LOG DISK THROTTLING] [START]", KPC(this),
"duration(s)", THROTTLING_DURATION_US / (1000 * 1000L), K(THROTTLING_CHUNK_SIZE));
PALF_LOG(INFO, "[LOG DISK THROTTLING] [START]", KPC(this), K(THROTTLING_CHUNK_SIZE));
}
}
} else {
if (throttling_options_.need_throttling()) {
PALF_LOG(INFO, "[LOG DISK THROTTLING] [STOP]", KPC(this),
"duration(s)", THROTTLING_DURATION_US / (1000 * 1000L), K(THROTTLING_CHUNK_SIZE));
PALF_LOG(INFO, "[LOG DISK THROTTLING] [STOP]", KPC(this), K(THROTTLING_CHUNK_SIZE));
clean_up_not_guarded_by_lock_();
stat_.stop_throttling();
}
@ -193,8 +192,7 @@ int LogWritingThrottle::update_throtting_options_guarded_by_lock_(IPalfEnvImpl *
}
} else {
if (throttling_options_.need_throttling()) {
PALF_LOG(INFO, "[LOG DISK THROTTLING] [STOP] no need throttling any more", KPC(this),
"duration(s)", THROTTLING_DURATION_US / (1000 * 1000L), K(THROTTLING_CHUNK_SIZE));
PALF_LOG(INFO, "[LOG DISK THROTTLING] [STOP] no need throttling any more", KPC(this), K(THROTTLING_CHUNK_SIZE));
clean_up_not_guarded_by_lock_();
stat_.stop_throttling();
}

View File

@ -120,7 +120,6 @@ private:
typedef common::ObSpinLockGuard SpinLockGuard;
static const int64_t UPDATE_INTERVAL_US = 500 * 1000L;//500ms
const int64_t DETECT_INTERVAL_US = 30 * 1000L;//30ms
const int64_t THROTTLING_DURATION_US = 1800 * 1000 * 1000L;//1800s
const int64_t THROTTLING_CHUNK_SIZE = MAX_LOG_BUFFER_SIZE;
//ts of lastest updating writing throttling info
int64_t last_update_ts_;

View File

@ -149,8 +149,11 @@ int PalfDiskOptionsWrapper::update_disk_options_not_guarded_by_lock_(const PalfD
}
//always update writing_throttling_trigger_percentage_
const int64_t new_trigger_percentage = disk_opts_for_recycling_blocks.log_disk_throttling_percentage_;
const int64_t new_maximum_duration = disk_opts_for_recycling_blocks.log_disk_throttling_maximum_duration_;
disk_opts_for_recycling_blocks_.log_disk_throttling_percentage_ = new_trigger_percentage;
disk_opts_for_stopping_writing_.log_disk_throttling_percentage_ = new_trigger_percentage;
disk_opts_for_recycling_blocks_.log_disk_throttling_maximum_duration_ = new_maximum_duration;
disk_opts_for_stopping_writing_.log_disk_throttling_maximum_duration_ = new_maximum_duration;
}
return ret;

View File

@ -124,6 +124,7 @@ public:
options.total_disk_space_ = disk_opts_for_stopping_writing_.log_disk_usage_limit_size_;
options.stopping_writing_percentage_ = disk_opts_for_stopping_writing_.log_disk_utilization_limit_threshold_;
options.trigger_percentage_ = disk_opts_for_stopping_writing_.log_disk_throttling_percentage_;
options.maximum_duration_ = disk_opts_for_stopping_writing_.log_disk_throttling_maximum_duration_;
options.unrecyclable_disk_space_ = cur_unrecyclable_log_disk_size_;
}

View File

@ -38,17 +38,22 @@ void PalfDiskOptions::reset()
log_disk_utilization_limit_threshold_ = -1;
log_disk_utilization_threshold_ = -1;
log_disk_throttling_percentage_ = -1;
log_disk_throttling_maximum_duration_ = -1;
log_writer_parallelism_ = -1;
}
bool PalfDiskOptions::is_valid() const
{
const int64_t MIN_DURATION = 1 * 1000 * 1000L;
const int64_t MAX_DURATION = 3 * 24 * 60 * 60 * 1000 * 1000L;
return -1 != log_disk_usage_limit_size_ && log_disk_usage_limit_size_ >= 4 * PALF_PHY_BLOCK_SIZE
&& 1 <=log_disk_utilization_threshold_ && 100 >= log_disk_utilization_threshold_
&& 1 <=log_disk_utilization_limit_threshold_ && 100 >= log_disk_utilization_limit_threshold_
&& log_disk_utilization_limit_threshold_ > log_disk_utilization_threshold_
&& log_disk_throttling_percentage_ >= MIN_WRITING_THTOTTLING_TRIGGER_PERCENTAGE
&& log_disk_throttling_percentage_ <= 100
&& log_disk_throttling_maximum_duration_ >= MIN_DURATION
&& log_disk_throttling_maximum_duration_ <= MAX_DURATION
&& log_writer_parallelism_ >= 1 && log_writer_parallelism_ <= 8;
}
@ -58,8 +63,8 @@ bool PalfDiskOptions::operator==(const PalfDiskOptions &palf_disk_options) const
&& log_disk_utilization_threshold_ == palf_disk_options.log_disk_utilization_threshold_
&& log_disk_utilization_limit_threshold_ == palf_disk_options.log_disk_utilization_limit_threshold_
&& log_disk_throttling_percentage_ == palf_disk_options.log_disk_throttling_percentage_
&& log_disk_throttling_maximum_duration_ == palf_disk_options.log_disk_throttling_maximum_duration_
&& log_writer_parallelism_ == palf_disk_options.log_writer_parallelism_;
}
PalfDiskOptions &PalfDiskOptions::operator=(const PalfDiskOptions &other)
@ -68,6 +73,7 @@ PalfDiskOptions &PalfDiskOptions::operator=(const PalfDiskOptions &other)
log_disk_utilization_threshold_ = other.log_disk_utilization_threshold_;
log_disk_utilization_limit_threshold_ = other.log_disk_utilization_limit_threshold_;
log_disk_throttling_percentage_ = other.log_disk_throttling_percentage_;
log_disk_throttling_maximum_duration_ = other.log_disk_throttling_maximum_duration_;
log_writer_parallelism_ = other.log_writer_parallelism_;
return *this;
}
@ -131,6 +137,7 @@ void PalfThrottleOptions::reset()
total_disk_space_ = -1;
stopping_writing_percentage_ = -1;
trigger_percentage_ = -1;
maximum_duration_ = -1;
unrecyclable_disk_space_ = -1;
}
@ -139,6 +146,7 @@ bool PalfThrottleOptions::is_valid() const
return (total_disk_space_ > 0
&& stopping_writing_percentage_ > 0 && stopping_writing_percentage_ <= 100
&& trigger_percentage_ >= MIN_WRITING_THTOTTLING_TRIGGER_PERCENTAGE && trigger_percentage_ <= 100
&& maximum_duration_ > 0
&& unrecyclable_disk_space_ >= 0);
}
@ -147,7 +155,9 @@ bool PalfThrottleOptions::operator==(const PalfThrottleOptions &other) const
return total_disk_space_ == other.total_disk_space_
&& stopping_writing_percentage_ == other.stopping_writing_percentage_
&& trigger_percentage_ == other.trigger_percentage_
&& maximum_duration_ == other.maximum_duration_
&& unrecyclable_disk_space_ == other.unrecyclable_disk_space_;
}
}
}
}// end of namespace palf
}// end of namespace oceanbase

View File

@ -31,6 +31,7 @@ struct PalfDiskOptions
log_disk_utilization_threshold_(-1),
log_disk_utilization_limit_threshold_(-1),
log_disk_throttling_percentage_(-1),
log_disk_throttling_maximum_duration_(-1),
log_writer_parallelism_(-1)
{}
~PalfDiskOptions() { reset(); }
@ -43,11 +44,13 @@ struct PalfDiskOptions
int log_disk_utilization_threshold_;
int log_disk_utilization_limit_threshold_;
int64_t log_disk_throttling_percentage_;
int64_t log_disk_throttling_maximum_duration_;
int log_writer_parallelism_;
TO_STRING_KV("log_disk_size(MB)", log_disk_usage_limit_size_ / MB,
"log_disk_utilization_threshold(%)", log_disk_utilization_threshold_,
"log_disk_utilization_limit_threshold(%)", log_disk_utilization_limit_threshold_,
"log_disk_throttling_percentage(%)", log_disk_throttling_percentage_,
"log_disk_throttling_maximum_duration(s)", log_disk_throttling_maximum_duration_ / (1000 * 1000),
"log_writer_parallelism", log_writer_parallelism_);
};
@ -193,14 +196,18 @@ struct PalfThrottleOptions
bool operator==(const PalfThrottleOptions &rhs) const;
// size of available log disk when writing throttling triggered
inline int64_t get_available_size_after_limit() const;
inline int64_t get_maximum_duration() const {return maximum_duration_;}
inline bool need_throttling() const;
static constexpr int64_t MB = 1024*1024ll;
TO_STRING_KV("total_disk_space", total_disk_space_ / MB, K_(stopping_writing_percentage),
K_(trigger_percentage), "unrecyclable_disk_space(MB)", unrecyclable_disk_space_ / MB);
TO_STRING_KV("total_disk_space", total_disk_space_ / MB,
K_(stopping_writing_percentage), K_(trigger_percentage),
"maximum_duration(s)", maximum_duration_/ (1000 * 1000L),
"unrecyclable_disk_space(MB)", unrecyclable_disk_space_ / MB);
public:
int64_t total_disk_space_;
int64_t stopping_writing_percentage_;
int64_t trigger_percentage_;
int64_t maximum_duration_;
int64_t unrecyclable_disk_space_;
};

View File

@ -716,6 +716,7 @@ int ObTenant::construct_mtl_init_ctx(const ObTenantMeta &meta, share::ObTenantMo
mtl_init_ctx_->palf_options_.disk_options_.log_disk_utilization_threshold_ = 80;
mtl_init_ctx_->palf_options_.disk_options_.log_disk_utilization_limit_threshold_ = 95;
mtl_init_ctx_->palf_options_.disk_options_.log_disk_throttling_percentage_ = 100;
mtl_init_ctx_->palf_options_.disk_options_.log_disk_throttling_maximum_duration_ = 2 * 60 * 60 * 1000 * 1000L;//2h
mtl_init_ctx_->palf_options_.disk_options_.log_writer_parallelism_ = 3;
ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID()));
if (OB_UNLIKELY(!tenant_config.is_valid())) {

View File

@ -9009,6 +9009,27 @@ int ObRootService::set_config_pre_hook(obrpc::ObAdminSetConfigArg &arg)
LOG_WARN("config invalid", KR(ret), K(*item), K(interval), K(tenant_id));
}
}
} else if (0 == STRCMP(item->name_.ptr(), LOG_DISK_UTILIZATION_LIMIT_THRESHOLD)) {
// check log_disk_utilization_limit_threshold
for (int i = 0; i < item->tenant_ids_.count() && valid; i++) {
valid = valid && ObConfigLogDiskLimitThresholdIntChecker::check(item->tenant_ids_.at(i), *item);
if (!valid) {
ret = OB_INVALID_ARGUMENT;
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "log_disk_utilization_limit_threshold should be greater than log_disk_throttling_percentage "
"when log_disk_throttling_percentage is not equal to 100");
LOG_WARN("config invalid", "item", *item, K(ret), K(i), K(item->tenant_ids_.at(i)));
}
}
} else if (0 == STRCMP(item->name_.ptr(), LOG_DISK_THROTTLING_PERCENTAGE)) {
// check log_disk_throttling_percentage
for (int i = 0; i < item->tenant_ids_.count() && valid; i++) {
valid = valid && ObConfigLogDiskThrottlingPercentageIntChecker::check(item->tenant_ids_.at(i), *item);
if (!valid) {
ret = OB_INVALID_ARGUMENT;
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "log_disk_throttling_percentage should be equal to 100 or smaller than log_disk_utilization_limit_threshold");
LOG_WARN("config invalid", "item", *item, K(ret), K(i), K(item->tenant_ids_.at(i)));
}
}
}
}
return ret;

View File

@ -125,6 +125,55 @@ int64_t ObConfigWriteThrottleTriggerIntChecker::get_freeze_trigger_percentage_(c
return percent;
}
bool ObConfigLogDiskLimitThresholdIntChecker::check(const uint64_t tenant_id,
const ObAdminSetConfigItem &t)
{
bool is_valid = false;
const int64_t value = ObConfigIntParser::get(t.value_.ptr(), is_valid);
const int64_t throttling_percentage = get_log_disk_throttling_percentage_(tenant_id);
if (is_valid) {
is_valid = (throttling_percentage != 0);
}
if (is_valid) {
is_valid = (throttling_percentage == 100) || (value > throttling_percentage);
}
return is_valid;
}
int64_t ObConfigLogDiskLimitThresholdIntChecker::get_log_disk_throttling_percentage_(const uint64_t tenant_id)
{
int64_t percent = 0;
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id));
if (tenant_config.is_valid()) {
percent = tenant_config->log_disk_throttling_percentage;
}
return percent;
}
bool ObConfigLogDiskThrottlingPercentageIntChecker::check(const uint64_t tenant_id, const obrpc::ObAdminSetConfigItem &t)
{
bool is_valid = false;
const int64_t value = ObConfigIntParser::get(t.value_.ptr(), is_valid);
const int64_t limit_threshold = get_log_disk_utilization_limit_threshold_(tenant_id);
if (is_valid) {
is_valid = (limit_threshold != 0);
}
if (is_valid) {
is_valid = (value == 100) || (value < limit_threshold);
}
return is_valid;
}
int64_t ObConfigLogDiskThrottlingPercentageIntChecker::get_log_disk_utilization_limit_threshold_(const uint64_t tenant_id)
{
int64_t threshold = 0;
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id));
if (tenant_config.is_valid()) {
threshold = tenant_config->log_disk_utilization_limit_threshold;
}
return threshold;
}
bool ObConfigTabletSizeChecker::check(const ObConfigItem &t) const
{
bool is_valid = false;

View File

@ -114,6 +114,28 @@ private:
DISALLOW_COPY_AND_ASSIGN(ObConfigWriteThrottleTriggerIntChecker);
};
//only used for RS checking
class ObConfigLogDiskLimitThresholdIntChecker
{
public:
static bool check(const uint64_t tenant_id,
const obrpc::ObAdminSetConfigItem &t);
private:
static int64_t get_log_disk_throttling_percentage_(const uint64_t tenant_id);
DISALLOW_COPY_AND_ASSIGN(ObConfigLogDiskLimitThresholdIntChecker);
};
//only used for RS checking
class ObConfigLogDiskThrottlingPercentageIntChecker
{
public:
static bool check(const uint64_t tenant_id,
const obrpc::ObAdminSetConfigItem &t);
private:
static int64_t get_log_disk_utilization_limit_threshold_(const uint64_t tenant_id);
DISALLOW_COPY_AND_ASSIGN(ObConfigLogDiskThrottlingPercentageIntChecker);
};
class ObConfigTabletSizeChecker
: public ObConfigChecker
{

View File

@ -58,6 +58,9 @@ const char* const COMPATIBLE = "compatible";
const char* const WEAK_READ_VERSION_REFRESH_INTERVAL = "weak_read_version_refresh_interval";
const char* const PARTITION_BALANCE_SCHEDULE_INTERVAL = "partition_balance_schedule_interval";
const char* const BALANCER_IDLE_TIME = "balancer_idle_time";
const char* const LOG_DISK_UTILIZATION_LIMIT_THRESHOLD = "log_disk_utilization_limit_threshold";
const char* const LOG_DISK_THROTTLING_PERCENTAGE = "log_disk_throttling_percentage";
class ObServerMemoryConfig;
class ObServerConfig : public ObCommonConfig

View File

@ -587,6 +587,10 @@ DEF_INT(log_disk_throttling_percentage, OB_TENANT_PARAMETER, "60",
"the threshold of the size of the log disk when writing_limit will be triggered. Rang:[40,100]. setting 100 means turn off writing limit",
ObParameterAttr(Section::LOGSERVICE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_TIME(log_disk_throttling_maximum_duration, OB_TENANT_PARAMETER, "2h", "[1s, 3d]",
"maximum duration of log disk throttling, that is the time remaining until the log disk space is exhausted after log disk throttling triggered.",
ObParameterAttr(Section::LOGSERVICE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_TIME(log_storage_warning_tolerance_time, OB_CLUSTER_PARAMETER, "5s",
"[1s,300s]",
"time to tolerate log disk io delay, after that, the disk status will be set warning. "

View File

@ -120,6 +120,7 @@ location_refresh_thread_count
log_archive_concurrency
log_disk_percentage
log_disk_size
log_disk_throttling_maximum_duration
log_disk_throttling_percentage
log_disk_utilization_limit_threshold
log_disk_utilization_threshold

View File

@ -78,6 +78,7 @@ TEST_F(TestPalfThrottling, test_palf_options)
int64_t utilization_limit_threshold = 95;
int64_t throttling_percentage = 60;
wrapper.disk_opts_for_stopping_writing_.log_disk_throttling_percentage_ = throttling_percentage;
wrapper.disk_opts_for_stopping_writing_.log_disk_throttling_maximum_duration_ = 7200 * 1000 * 1000L;
wrapper.disk_opts_for_stopping_writing_.log_disk_usage_limit_size_ = total_disk_size;
wrapper.disk_opts_for_stopping_writing_.log_disk_utilization_threshold_ = 80;
wrapper.disk_opts_for_stopping_writing_.log_disk_utilization_limit_threshold_ = utilization_limit_threshold;
@ -140,6 +141,7 @@ TEST_F(TestPalfThrottling, test_log_write_throttle)
PalfEnvImpl palf_env_impl;
palf_env_impl.is_inited_ = true;
palf_env_impl.disk_options_wrapper_.disk_opts_for_stopping_writing_.log_disk_throttling_percentage_ = throttling_percentage;
palf_env_impl.disk_options_wrapper_.disk_opts_for_stopping_writing_.log_disk_throttling_maximum_duration_ = 7200 * 1000 * 1000L;
palf_env_impl.disk_options_wrapper_.disk_opts_for_stopping_writing_.log_disk_usage_limit_size_ = total_disk_size;
palf_env_impl.disk_options_wrapper_.disk_opts_for_stopping_writing_.log_disk_utilization_threshold_ = 80;
palf_env_impl.disk_options_wrapper_.disk_opts_for_stopping_writing_.log_disk_utilization_limit_threshold_ = utilization_limit_threshold;
@ -205,6 +207,7 @@ TEST_F(TestPalfThrottling, test_log_write_throttle)
throttle.update_throttling_options(&palf_env_impl);
throttle.throttling(1024, g_need_purging_throttling_func, &palf_env_impl);
palf_env_impl.disk_options_wrapper_.get_throttling_options(throttle_options);
PALF_LOG(INFO, "case 4: YYY test need throttling", K(throttle_options), K(throttle.throttling_options_));
ASSERT_EQ(throttle_options, throttle.throttling_options_);
ASSERT_EQ(true, throttle.need_throttling_not_guarded_by_lock_(g_need_purging_throttling_func));
ASSERT_EQ(true, throttle.stat_.has_ever_throttled());