1247 lines
		
	
	
		
			49 KiB
		
	
	
	
		
			C++
		
	
	
	
	
	
			
		
		
	
	
			1247 lines
		
	
	
		
			49 KiB
		
	
	
	
		
			C++
		
	
	
	
	
	
| /**
 | |
|  * 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 "palf_env_impl.h"
 | |
| #include <string.h>
 | |
| #include "lib/lock/ob_spin_lock.h"
 | |
| #include "lib/ob_define.h"
 | |
| #include "lib/ob_errno.h"
 | |
| #include "lib/oblog/ob_log.h"
 | |
| #include "lib/time/ob_time_utility.h"
 | |
| #include "lib/utility/ob_macro_utils.h"
 | |
| #include "lib/oblog/ob_log_module.h"
 | |
| #include "share/allocator/ob_tenant_mutil_allocator.h"
 | |
| #include "share/config/ob_server_config.h"
 | |
| #include "share/ob_errno.h"
 | |
| #include "share/ob_occam_thread_pool.h"
 | |
| #include "log_define.h"
 | |
| #include "palf_handle_impl_guard.h"             // IPalfHandleImplGuard
 | |
| #include "palf_handle.h"
 | |
| #include "log_loop_thread.h"
 | |
| #include "log_rpc.h"
 | |
| #include "log_block_pool_interface.h"
 | |
| 
 | |
| namespace oceanbase
 | |
| {
 | |
| using namespace common;
 | |
| using namespace share;
 | |
| namespace palf
 | |
| {
 | |
| PalfHandleImpl *PalfHandleImplFactory::alloc()
 | |
| {
 | |
|   return MTL_NEW(PalfHandleImpl, "palf_env");
 | |
| }
 | |
| 
 | |
| void PalfHandleImplFactory::free(IPalfHandleImpl *palf_handle_impl)
 | |
| {
 | |
|   MTL_DELETE(IPalfHandleImpl, "palf_env", palf_handle_impl);
 | |
| }
 | |
| 
 | |
| PalfHandleImpl *PalfHandleImplAlloc::alloc_value()
 | |
| {
 | |
|   return NULL;
 | |
| }
 | |
| 
 | |
| void PalfHandleImplAlloc::free_value(IPalfHandleImpl *palf_handle_impl)
 | |
| {
 | |
|   PalfHandleImplFactory::free(palf_handle_impl);
 | |
|   palf_handle_impl = NULL;
 | |
| }
 | |
| 
 | |
| PalfHandleImplAlloc::Node *PalfHandleImplAlloc::alloc_node(IPalfHandleImpl *palf_handle_impl)
 | |
| {
 | |
|   UNUSED(palf_handle_impl);
 | |
|   return op_reclaim_alloc(Node);
 | |
| }
 | |
| 
 | |
| void PalfHandleImplAlloc::free_node(PalfHandleImplAlloc::Node *node)
 | |
| {
 | |
|   op_reclaim_free(node);
 | |
|   node = NULL;
 | |
| }
 | |
| 
 | |
| int PalfDiskOptionsWrapper::init(const PalfDiskOptions &disk_opts)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (false == disk_opts.is_valid()) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|   } else {
 | |
|     disk_opts_for_recycling_blocks_ = disk_opts_for_stopping_writing_ = disk_opts;
 | |
|     status_ = Status::NORMAL_STATUS;
 | |
|     cur_unrecyclable_log_disk_size_ = 0;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| void PalfDiskOptionsWrapper::reset()
 | |
| {
 | |
|   ObSpinLockGuard guard(disk_opts_lock_);
 | |
|   disk_opts_for_recycling_blocks_.reset();
 | |
|   disk_opts_for_stopping_writing_.reset();
 | |
|   status_ = Status::INVALID_STATUS;
 | |
|   cur_unrecyclable_log_disk_size_ = -1;
 | |
| }
 | |
| 
 | |
| int PalfDiskOptionsWrapper::update_disk_options(const PalfDiskOptions &disk_opts_for_recycling_blocks)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   ObSpinLockGuard guard(disk_opts_lock_);
 | |
|   return update_disk_options_not_guarded_by_lock_(disk_opts_for_recycling_blocks);
 | |
| }
 | |
| 
 | |
| void PalfDiskOptionsWrapper::set_cur_unrecyclable_log_disk_size(const int64_t unrecyclable_log_disk_size)
 | |
| {
 | |
|   ObSpinLockGuard guard(disk_opts_lock_);
 | |
|   cur_unrecyclable_log_disk_size_ = unrecyclable_log_disk_size;
 | |
| }
 | |
| 
 | |
| bool PalfDiskOptionsWrapper::need_throttling() const
 | |
| {
 | |
|   bool is_need = false;
 | |
|   ObSpinLockGuard guard(disk_opts_lock_);
 | |
|   const int64_t trigger_size = disk_opts_for_stopping_writing_.log_disk_usage_limit_size_ * disk_opts_for_stopping_writing_.log_disk_throttling_percentage_ / 100;
 | |
|   return disk_opts_for_stopping_writing_.is_valid() && cur_unrecyclable_log_disk_size_ > trigger_size;
 | |
| }
 | |
| 
 | |
| void PalfDiskOptionsWrapper::change_to_normal()
 | |
| {
 | |
|   ObSpinLockGuard guard(disk_opts_lock_);
 | |
|   status_ = Status::NORMAL_STATUS;
 | |
|   disk_opts_for_stopping_writing_ = disk_opts_for_recycling_blocks_;
 | |
|   PALF_LOG(INFO, "change_to_normal", KPC(this));
 | |
| }
 | |
| 
 | |
| int PalfDiskOptionsWrapper::update_disk_options_not_guarded_by_lock_(const PalfDiskOptions &disk_opts_for_recycling_blocks)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int64_t curr_stop_write_limit_size =
 | |
|     disk_opts_for_stopping_writing_.log_disk_usage_limit_size_ * disk_opts_for_stopping_writing_.log_disk_utilization_limit_threshold_;
 | |
|   int64_t next_stop_write_limit_size =
 | |
|     disk_opts_for_recycling_blocks.log_disk_usage_limit_size_ * disk_opts_for_recycling_blocks.log_disk_utilization_limit_threshold_;
 | |
|   if (false == disk_opts_for_recycling_blocks.is_valid()) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|   } else {
 | |
|     if (Status::SHRINKING_STATUS == status_) {
 | |
|       ret = OB_STATE_NOT_MATCH;
 | |
|       PALF_LOG(WARN, "don't support shrink log disk concurrently", K(ret), KPC(this));
 | |
|     } else if (disk_opts_for_recycling_blocks_ == disk_opts_for_recycling_blocks) {
 | |
|       PALF_LOG(INFO, "no need update disk options", K(ret), K(disk_opts_for_recycling_blocks_), K(disk_opts_for_recycling_blocks));
 | |
|     } else if (curr_stop_write_limit_size > next_stop_write_limit_size) {
 | |
|       status_ = Status::SHRINKING_STATUS;
 | |
|       // In process of shrinking, to avoid stopping writing,
 | |
|       // 'disk_opts_for_stopping_writing_' is still an original value, update it
 | |
|       // with 'disk_opts_for_recycling_blocks' until there is no possibility
 | |
|       // caused stopping writing.
 | |
|       disk_opts_for_recycling_blocks_ = disk_opts_for_recycling_blocks;
 | |
|     } else {
 | |
|       status_ = Status::NORMAL_STATUS;
 | |
|       disk_opts_for_recycling_blocks_ = disk_opts_for_stopping_writing_ = disk_opts_for_recycling_blocks;
 | |
|       PALF_LOG(INFO, "update_disk_options_not_guarded_by_lock_ success", K(curr_stop_write_limit_size), K(next_stop_write_limit_size));
 | |
|     }
 | |
|     //always update writing_throttling_trigger_percentage_
 | |
|     const int64_t new_trigger_percentage = disk_opts_for_recycling_blocks.log_disk_throttling_percentage_;
 | |
|     disk_opts_for_recycling_blocks_.log_disk_throttling_percentage_ = new_trigger_percentage;
 | |
|     disk_opts_for_stopping_writing_.log_disk_throttling_percentage_ = new_trigger_percentage;
 | |
| 
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| PalfEnvImpl::PalfEnvImpl() : palf_meta_lock_(common::ObLatchIds::PALF_ENV_LOCK),
 | |
|                              log_alloc_mgr_(NULL),
 | |
|                              log_block_pool_(NULL),
 | |
|                              fetch_log_engine_(),
 | |
|                              log_rpc_(),
 | |
|                              cb_thread_pool_(),
 | |
|                              log_io_worker_wrapper_(),
 | |
|                              block_gc_timer_task_(),
 | |
|                              log_updater_(),
 | |
|                              monitor_(NULL),
 | |
|                              disk_options_wrapper_(),
 | |
|                              check_disk_print_log_interval_(OB_INVALID_TIMESTAMP),
 | |
|                              self_(),
 | |
|                              palf_handle_impl_map_(64),  // 指定min_size=64
 | |
|                              last_palf_epoch_(0),
 | |
|                              rebuild_replica_log_lag_threshold_(0),
 | |
|                              diskspace_enough_(true),
 | |
|                              tenant_id_(0),
 | |
|                              is_inited_(false),
 | |
|                              is_running_(false)
 | |
| {
 | |
|   log_dir_[0] = '\0';
 | |
|   tmp_log_dir_[0] = '\0';
 | |
| }
 | |
| 
 | |
| PalfEnvImpl::~PalfEnvImpl()
 | |
| {
 | |
|   destroy();
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::init(
 | |
|     const PalfOptions &options,
 | |
|     const char *base_dir, const ObAddr &self,
 | |
|     const int64_t cluster_id,
 | |
|     const int64_t tenant_id,
 | |
|     rpc::frame::ObReqTransport *transport,
 | |
|     common::ObILogAllocator *log_alloc_mgr,
 | |
|     ILogBlockPool *log_block_pool,
 | |
|     PalfMonitorCb *monitor)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   // TODO by runlin: configurable
 | |
|   log_io_worker_config_.io_worker_num_ = 1;
 | |
|   log_io_worker_config_.io_queue_capcity_ = 100 * 1024;
 | |
|   log_io_worker_config_.batch_width_ = 8;
 | |
|   log_io_worker_config_.batch_depth_ = PALF_SLIDING_WINDOW_SIZE;
 | |
|   const int64_t io_cb_num = PALF_SLIDING_WINDOW_SIZE * 128;
 | |
|   if (is_inited_) {
 | |
|     ret = OB_INIT_TWICE;
 | |
|     PALF_LOG(ERROR, "PalfEnvImpl is inited twiced", K(ret));
 | |
|   } else if (OB_ISNULL(base_dir) || !self.is_valid() || NULL == transport
 | |
|              || OB_ISNULL(log_alloc_mgr) || OB_ISNULL(log_block_pool) || OB_ISNULL(monitor)) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     PALF_LOG(ERROR, "invalid arguments", K(ret), KP(transport), K(base_dir), K(self), KP(transport),
 | |
|              KP(log_alloc_mgr), KP(log_block_pool), KP(monitor));
 | |
|   } else if (OB_FAIL(fetch_log_engine_.init(this, log_alloc_mgr))) {
 | |
|     PALF_LOG(ERROR, "FetchLogEngine init failed", K(ret));
 | |
|   } else if (OB_FAIL(log_rpc_.init(self, cluster_id, tenant_id, transport))) {
 | |
|     PALF_LOG(ERROR, "LogRpc init failed", K(ret));
 | |
|   } else if (OB_FAIL(cb_thread_pool_.init(io_cb_num, this))) {
 | |
|     PALF_LOG(ERROR, "LogIOTaskThreadPool init failed", K(ret));
 | |
|   } else if (OB_FAIL(log_io_worker_wrapper_.init(log_io_worker_config_,
 | |
|                                                  tenant_id,
 | |
|                                                  cb_thread_pool_.get_tg_id(),
 | |
|                                                  log_alloc_mgr, this))) {
 | |
|     PALF_LOG(ERROR, "LogIOWorker init failed", K(ret));
 | |
|   } else if (OB_FAIL(block_gc_timer_task_.init(this))) {
 | |
|     PALF_LOG(ERROR, "ObCheckLogBlockCollectTask init failed", K(ret));
 | |
|   } else if ((pret = snprintf(log_dir_, MAX_PATH_SIZE, "%s", base_dir)) && false) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|   } else if ((pret = snprintf(tmp_log_dir_, MAX_PATH_SIZE, "%s/tmp_dir", log_dir_)) && false) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "error unexpected", K(ret));
 | |
|   } else if (pret < 0 || pret >= MAX_PATH_SIZE) {
 | |
|     ret = OB_BUF_NOT_ENOUGH;
 | |
|     PALF_LOG(ERROR, "construct log path failed", K(ret), K(pret));
 | |
|   } else if (OB_FAIL(palf_handle_impl_map_.init("LOG_HASH_MAP", tenant_id))) {
 | |
|     PALF_LOG(ERROR, "palf_handle_impl_map_ init failed", K(ret));
 | |
|   } else if (OB_FAIL(log_loop_thread_.init(this))) {
 | |
|     PALF_LOG(ERROR, "log_loop_thread_ init failed", K(ret));
 | |
|   } else if (OB_FAIL(
 | |
|                  election_timer_.init_and_start(1, 10_ms, "ElectTimer"))) { // just one worker thread
 | |
|     PALF_LOG(ERROR, "election_timer_ init failed", K(ret));
 | |
|   } else if (OB_FAIL(election::GLOBAL_INIT_ELECTION_MODULE())) {
 | |
|     PALF_LOG(ERROR, "global init election module failed", K(ret));
 | |
|   } else if (OB_FAIL(disk_options_wrapper_.init(options.disk_options_))) {
 | |
|     PALF_LOG(ERROR, "disk_options_wrapper_ init failed", K(ret));
 | |
|   } else if (OB_FAIL(log_updater_.init(this))) {
 | |
|     PALF_LOG(ERROR, "LogUpdater init failed", K(ret));
 | |
|   } else {
 | |
|     log_alloc_mgr_ = log_alloc_mgr;
 | |
|     log_block_pool_ = log_block_pool;
 | |
|     monitor_ = monitor;
 | |
|     self_ = self;
 | |
|     tenant_id_ = tenant_id;
 | |
|     is_inited_ = true;
 | |
|     is_running_ = true;
 | |
|     PALF_LOG(INFO, "PalfEnvImpl init success", K(ret), K(self_), KPC(this));
 | |
|   }
 | |
|   if (OB_FAIL(ret) && OB_INIT_TWICE != ret) {
 | |
|     destroy();
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::start()
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else if (OB_FAIL(scan_all_palf_handle_impl_director_())) {
 | |
|     PALF_LOG(WARN, "scan_all_palf_handle_impl_director_ failed", K(ret));
 | |
|   } else if (OB_FAIL(cb_thread_pool_.start())) {
 | |
|     PALF_LOG(ERROR, "LogIOTaskThreadPool start failed", K(ret));
 | |
|   } else if (OB_FAIL(log_io_worker_wrapper_.start())) {
 | |
|     PALF_LOG(ERROR, "LogIOWorker start failed", K(ret));
 | |
|   } else if (OB_FAIL(block_gc_timer_task_.start())) {
 | |
|     PALF_LOG(ERROR, "FileCollectTimerTask start failed", K(ret));
 | |
| 	} else if (OB_FAIL(fetch_log_engine_.start())) {
 | |
|     PALF_LOG(ERROR, "FetchLogEngine start failed", K(ret));
 | |
|   } else if (OB_FAIL(log_loop_thread_.start())) {
 | |
|     PALF_LOG(ERROR, "log_loop_thread_ start failed", K(ret));
 | |
|   } else if (OB_FAIL(log_updater_.start())) {
 | |
|     PALF_LOG(ERROR, "LogUpdater start failed", K(ret));
 | |
|   } else {
 | |
|     is_running_ = true;
 | |
|     PALF_LOG(INFO, "PalfEnv start success", K(ret));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| void PalfEnvImpl::stop()
 | |
| {
 | |
|   if (is_running_) {
 | |
|     PALF_LOG(INFO, "PalfEnvImpl begin stop", KPC(this));
 | |
|     is_running_ = false;
 | |
|     log_io_worker_wrapper_.stop();
 | |
|     cb_thread_pool_.stop();
 | |
|     block_gc_timer_task_.stop();
 | |
|     fetch_log_engine_.stop();
 | |
|     log_loop_thread_.stop();
 | |
|     log_updater_.stop();
 | |
|     PALF_LOG(INFO, "PalfEnvImpl stop success", KPC(this));
 | |
|   }
 | |
| }
 | |
| 
 | |
| void PalfEnvImpl::wait()
 | |
| {
 | |
|   PALF_LOG(INFO, "PalfEnvImpl begin wait", KPC(this));
 | |
|   log_io_worker_wrapper_.wait();
 | |
|   cb_thread_pool_.wait();
 | |
|   block_gc_timer_task_.wait();
 | |
|   fetch_log_engine_.wait();
 | |
|   log_loop_thread_.wait();
 | |
|   log_updater_.wait();
 | |
|   PALF_LOG(INFO, "PalfEnvImpl wait success", KPC(this));
 | |
| }
 | |
| 
 | |
| void PalfEnvImpl::destroy()
 | |
| {
 | |
|   PALF_LOG_RET(WARN, OB_SUCCESS, "PalfEnvImpl destroy", KPC(this));
 | |
|   is_running_ = false;
 | |
|   is_inited_ = false;
 | |
|   palf_handle_impl_map_.destroy();
 | |
|   log_io_worker_wrapper_.destroy();
 | |
|   cb_thread_pool_.destroy();
 | |
|   log_loop_thread_.destroy();
 | |
|   block_gc_timer_task_.destroy();
 | |
|   fetch_log_engine_.destroy();
 | |
|   log_updater_.destroy();
 | |
|   log_rpc_.destroy();
 | |
|   log_alloc_mgr_ = NULL;
 | |
|   monitor_ = NULL;
 | |
|   self_.reset();
 | |
|   log_dir_[0] = '\0';
 | |
|   tmp_log_dir_[0] = '\0';
 | |
|   disk_options_wrapper_.reset();
 | |
|   rebuild_replica_log_lag_threshold_ = 0;
 | |
| }
 | |
| 
 | |
| // NB: not thread safe
 | |
| int PalfEnvImpl::create_palf_handle_impl(const int64_t palf_id,
 | |
|                                          const AccessMode &access_mode,
 | |
|                                          const PalfBaseInfo &palf_base_info,
 | |
|                                          IPalfHandleImpl *&palf_handle_impl)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   WLockGuard guard(palf_meta_lock_);
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|     PALF_LOG(WARN, "PalfEnvImpl is not inited", K(ret));
 | |
|   } else if (OB_FAIL(create_palf_handle_impl_(palf_id, access_mode, palf_base_info,
 | |
|           NORMAL_REPLICA, palf_handle_impl))) {
 | |
|     palf_handle_impl = NULL;
 | |
|   } else {
 | |
|     PALF_LOG(INFO, "PalfEnvImpl create_palf_handle_impl finished", K(ret), K(palf_id), K(access_mode),
 | |
|         K(palf_base_info), KPC(this));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::create_palf_handle_impl_(const int64_t palf_id,
 | |
|                                           const AccessMode &access_mode,
 | |
|                                           const PalfBaseInfo &palf_base_info,
 | |
|                                           const LogReplicaType replica_type,
 | |
|                                           IPalfHandleImpl *&ipalf_handle_impl)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   char base_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   PalfHandleImpl *palf_handle_impl = NULL;
 | |
|   LSKey hash_map_key(palf_id);
 | |
|   const int64_t palf_epoch = ATOMIC_AAF(&last_palf_epoch_, 1);
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|     PALF_LOG(WARN, "PalfEnvImpl is not running", K(ret));
 | |
|   } else if (!is_running_) {
 | |
|     ret = OB_NOT_RUNNING;
 | |
|     PALF_LOG(WARN, "PalfEnvImpl is not running", K(ret));
 | |
|   } else if (false == is_valid_palf_id(palf_id)) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     PALF_LOG(WARN, "invalid arguments", K(ret), K(palf_id));
 | |
|   } else if (OB_ENTRY_EXIST == palf_handle_impl_map_.contains_key(hash_map_key)) {
 | |
|     ret = OB_ENTRY_EXIST;
 | |
|     PALF_LOG(WARN, "palf_handle has exist, ignore this request", K(ret), K(palf_id));
 | |
|   } else if (false == check_can_create_palf_handle_impl_()) {
 | |
|     ret = OB_LOG_OUTOF_DISK_SPACE;
 | |
|     PALF_LOG(ERROR, "PalfEnv can not hold more instance", K(ret), KPC(this), K(palf_id));
 | |
|   } else if (0 > (pret = snprintf(base_dir, MAX_PATH_SIZE, "%s/%ld", log_dir_, palf_id))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprinf failed", K(pret), K(palf_id));
 | |
|   } else if (OB_FAIL(create_directory(base_dir))) {
 | |
|     PALF_LOG(WARN, "prepare_directory_for_creating_ls failed!!!", K(ret), K(palf_id));
 | |
|   } else if (NULL == (palf_handle_impl = PalfHandleImplFactory::alloc())) {
 | |
|     ret = OB_ALLOCATE_MEMORY_FAILED;
 | |
|     PALF_LOG(WARN, "alloc palf_handle_impl failed", K(ret));
 | |
|   } else if (OB_FAIL(palf_handle_impl->init(palf_id, access_mode, palf_base_info, replica_type,
 | |
|       &fetch_log_engine_, base_dir, log_alloc_mgr_, log_block_pool_, &log_rpc_,
 | |
|       log_io_worker_wrapper_.get_log_io_worker(palf_id), this, self_, &election_timer_, palf_epoch))) {
 | |
|     PALF_LOG(ERROR, "IPalfHandleImpl init failed", K(ret), K(palf_id));
 | |
|     // NB: always insert value into hash map finally.
 | |
|   } else if (OB_FAIL(palf_handle_impl_map_.insert_and_get(hash_map_key, palf_handle_impl))) {
 | |
|     PALF_LOG(WARN, "palf_handle_impl_map_ insert_and_get failed", K(ret), K(palf_id));
 | |
|   } else if (OB_FAIL(palf_handle_impl->set_monitor_cb(monitor_))) {
 | |
|     PALF_LOG(WARN, "set_monitor_cb failed", K(ret), K(palf_id), KP_(monitor));
 | |
|   } else {
 | |
|     palf_handle_impl->set_scan_disk_log_finished();
 | |
|     ipalf_handle_impl = palf_handle_impl;
 | |
|   }
 | |
| 
 | |
|   if (OB_FAIL(ret) && NULL != palf_handle_impl) {
 | |
|     // if 'palf_handle_impl' has not been inserted into hash map,
 | |
|     // need reclaim manually.
 | |
|     PalfHandleImplFactory::free(palf_handle_impl);
 | |
|     palf_handle_impl = NULL;
 | |
|     if (OB_ENTRY_NOT_EXIST == palf_handle_impl_map_.contains_key(hash_map_key)) {
 | |
|       remove_directory(base_dir);
 | |
|     }
 | |
|   }
 | |
| 
 | |
|   PALF_LOG(INFO, "PalfEnvImpl create_palf_handle_impl_ finished", K(ret), K(palf_id),
 | |
|       K(access_mode), K(palf_base_info), K(replica_type), KPC(this));
 | |
| 
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::remove_palf_handle_impl(const int64_t palf_id)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   WLockGuard guard(palf_meta_lock_);
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|     PALF_LOG(ERROR, "PalfEnvImpl is not inited", K(ret));
 | |
|   } else if (OB_FAIL(remove_palf_handle_impl_from_map_not_guarded_by_lock_(palf_id))) {
 | |
|     PALF_LOG(WARN, "palf instance not exist", K(ret), KPC(this), K(palf_id));
 | |
|   } else if (OB_FAIL(wait_until_reference_count_to_zero_(palf_id))) {
 | |
|     PALF_LOG(WARN, "wait_until_reference_count_to_zero_ failed", K(ret), KPC(this), K(palf_id));
 | |
|   } else {
 | |
|   }
 | |
|   if (OB_ENTRY_NOT_EXIST == ret) {
 | |
|     ret = OB_SUCCESS;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_palf_handle_impl(const int64_t palf_id,
 | |
|                                       IPalfHandleImplGuard &palf_handle_impl_guard)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   IPalfHandleImpl *palf_handle_impl = NULL;
 | |
|   if (OB_FAIL(get_palf_handle_impl(palf_id, palf_handle_impl))) {
 | |
|     PALF_LOG(TRACE, "get_palf_handle_impl failed", K(ret), K(palf_id));
 | |
|   } else {
 | |
|     palf_handle_impl_guard.palf_env_impl_ = this;
 | |
|     palf_handle_impl_guard.palf_handle_impl_ = palf_handle_impl;
 | |
|     palf_handle_impl_guard.palf_id_ = palf_id;
 | |
|     PALF_LOG(TRACE, "get_palf_handle_impl success", K(palf_id), K(palf_handle_impl_guard));
 | |
|     // do nothing
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_palf_handle_impl(const int64_t palf_id,
 | |
|                                       IPalfHandleImpl *&ipalf_handle_impl)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   LSKey hash_map_key(palf_id);
 | |
|   IPalfHandleImpl *palf_handle_impl = NULL;
 | |
|   if (false == is_valid_palf_id(palf_id)) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     PALF_LOG(ERROR, "Invalid argument!!!", K(ret), K(palf_id));
 | |
|   } else if (OB_FAIL(palf_handle_impl_map_.get(hash_map_key, palf_handle_impl))) {
 | |
|     PALF_LOG(TRACE, "get from map failed", K(ret), K(palf_id), K(palf_handle_impl));
 | |
|   } else if (false == palf_handle_impl->check_can_be_used()) {
 | |
|     ret = OB_ENTRY_NOT_EXIST;
 | |
|   } else {
 | |
|     ipalf_handle_impl = palf_handle_impl;
 | |
|   }
 | |
| 
 | |
|   if (OB_FAIL(ret) && NULL != palf_handle_impl) {
 | |
|     revert_palf_handle_impl(palf_handle_impl);
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| void PalfEnvImpl::revert_palf_handle_impl(IPalfHandleImpl *ipalf_handle_impl)
 | |
| {
 | |
|   if (NULL != ipalf_handle_impl) {
 | |
|     palf_handle_impl_map_.revert(ipalf_handle_impl);
 | |
|   }
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::scan_all_palf_handle_impl_director_()
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   ObTimeGuard guard("PalfEnvImplStart", 0);
 | |
| 	// TODO by runlin: how to avoid modify 'log_disk_usage_limit_size_' after restart?
 | |
|   ReloadPalfHandleImplFunctor functor(this);
 | |
|   if (OB_FAIL(scan_dir(log_dir_, functor))) {
 | |
|     PALF_LOG(WARN, "scan_dir failed", K(ret));
 | |
|   } else {
 | |
|     guard.click("scan_dir");
 | |
|     PALF_LOG(INFO, "scan_all_palf_handle_impl_director_ success", K(ret), K(log_dir_), K(guard));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::create_directory(const char *base_dir)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   const mode_t mode = S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH;
 | |
|   char tmp_base_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   char log_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   char meta_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   if (0 > (pret = snprintf(tmp_base_dir, MAX_PATH_SIZE, "%s%s", base_dir, TMP_SUFFIX))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprinf failed", K(pret), K(base_dir));
 | |
|   } else if (0 > (pret = snprintf(log_dir, MAX_PATH_SIZE, "%s/log", tmp_base_dir))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprinf failed", K(pret), K(base_dir));
 | |
|   } else if (0 > (pret = snprintf(meta_dir, MAX_PATH_SIZE, "%s/meta", tmp_base_dir))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprinf failed", K(pret), K(base_dir));
 | |
|   } else if (-1 == (::mkdir(tmp_base_dir, mode))) {
 | |
|     ret = convert_sys_errno();
 | |
|     PALF_LOG(WARN, "mkdir failed", K(ret), K(errno), K(tmp_base_dir), K(base_dir));
 | |
|   } else if (-1 == (::mkdir(log_dir, mode))) {
 | |
|     ret = convert_sys_errno();
 | |
|     PALF_LOG(WARN, "mkdir failed", K(ret), K(errno), K(tmp_base_dir), K(base_dir));
 | |
|   } else if (-1 == (::mkdir(meta_dir, mode))) {
 | |
|     ret = convert_sys_errno();
 | |
|     PALF_LOG(WARN, "mkdir failed", K(ret), K(errno), K(tmp_base_dir), K(base_dir));
 | |
|   } else if (-1 == (::rename(tmp_base_dir, base_dir))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "rename tmp dir to normal dir failed", K(ret), K(errno), K(tmp_base_dir), K(base_dir));
 | |
|   } else if (OB_FAIL(FileDirectoryUtils::fsync_dir(log_dir_))) {
 | |
|     PALF_LOG(ERROR, "fsync_dir failed", K(ret), K(errno), K(tmp_base_dir), K(base_dir));
 | |
|   } else {
 | |
|     PALF_LOG(INFO, "prepare_directory_for_creating_ls success", K(ret), K(base_dir));
 | |
|   }
 | |
|   if (OB_FAIL(ret)) {
 | |
|     remove_directory_rec(tmp_base_dir, log_block_pool_);
 | |
|     remove_directory_rec(base_dir, log_block_pool_);
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| // step:
 | |
| // 1. rename log directory to tmp directory.
 | |
| // 2. delete tmp directory.
 | |
| // NB: '%s.tmp' is invalid block or invalid directory, before the restart phash of PalfEnvImpl,
 | |
| //     need delete thses tmp block or directory.
 | |
| int PalfEnvImpl::remove_directory(const char *log_dir)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   char tmp_log_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   if (0 > (pret = snprintf(tmp_log_dir, MAX_PATH_SIZE, "%s%s", log_dir, TMP_SUFFIX))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprintf failed", K(ret), K(pret), K(log_dir), K(tmp_log_dir));
 | |
|   } else if (-1 == ::rename(log_dir, tmp_log_dir)) {
 | |
|     ret = convert_sys_errno();
 | |
|     PALF_LOG(ERROR, "rename log dir to tmp dir failed", K(ret), K(errno), K(tmp_log_dir), K(log_dir));
 | |
|   } else if (OB_FAIL(remove_directory_rec(tmp_log_dir, log_block_pool_))) {
 | |
|     PALF_LOG(WARN, "remove_directory_rec failed", K(ret), K(log_dir), K(errno));
 | |
|   } else if (OB_FAIL(FileDirectoryUtils::fsync_dir(log_dir_))) {
 | |
|     PALF_LOG(ERROR, "fsync_dir failed", K(ret), KPC(this), K(log_dir));
 | |
|   } else {
 | |
|     PALF_LOG(WARN, "remove success", K(ret), K(log_dir));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| PalfEnvImpl::LogGetRecycableFileCandidate::LogGetRecycableFileCandidate()
 | |
|   : id_(-1),
 | |
|     min_block_id_(LOG_INVALID_BLOCK_ID),
 | |
|     min_block_max_scn_(),
 | |
|     min_using_block_id_(LOG_INVALID_BLOCK_ID),
 | |
|     oldest_palf_id_(INVALID_PALF_ID),
 | |
|     oldest_block_scn_(),
 | |
|     ret_code_(OB_SUCCESS)
 | |
| {}
 | |
| 
 | |
| PalfEnvImpl::LogGetRecycableFileCandidate::~LogGetRecycableFileCandidate()
 | |
| {
 | |
|   ret_code_ = OB_SUCCESS;
 | |
|   min_using_block_id_ = LOG_INVALID_BLOCK_ID;
 | |
|   min_block_max_scn_.reset();
 | |
|   min_block_id_ = LOG_INVALID_BLOCK_ID;
 | |
|   oldest_palf_id_ = INVALID_PALF_ID;
 | |
|   oldest_block_scn_.reset();
 | |
|   id_ = -1;
 | |
| }
 | |
| 
 | |
| bool PalfEnvImpl::LogGetRecycableFileCandidate::operator()(const LSKey &palf_id, IPalfHandleImpl *palf_handle_impl)
 | |
| {
 | |
|   bool bool_ret = true;
 | |
|   if (NULL == palf_handle_impl) {
 | |
|     PALF_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "the value in hashmap is NULL, unexpected error", K(palf_id), KP(palf_handle_impl));
 | |
|     bool_ret = false;
 | |
|   } else {
 | |
|     int ret = OB_SUCCESS;
 | |
|     const LSN base_lsn = palf_handle_impl->get_base_lsn_used_for_block_gc();
 | |
|     const block_id_t min_using_block_id = lsn_2_block(base_lsn, PALF_BLOCK_SIZE);
 | |
|     block_id_t min_block_id = LOG_INVALID_BLOCK_ID;
 | |
|     SCN min_block_max_scn;
 | |
|     // OB_ENTRY_NOT_EXIST means there is not any block;
 | |
|     // OB_NO_SUCH_FILE_OR_DIRECTORY means there is concurrently with rebuild.
 | |
|     // OB_ERR_OUT_OF_UPPER_BOUND means there is one block
 | |
|     auto need_skip_by_ret = [](const int ret ){
 | |
|       return OB_ENTRY_NOT_EXIST == ret  || OB_NO_SUCH_FILE_OR_DIRECTORY == ret
 | |
|           || OB_ERR_OUT_OF_UPPER_BOUND == ret;
 | |
|     };
 | |
|     if (false == base_lsn.is_valid()) {
 | |
|       PALF_LOG(WARN, "base_lsn is invalid", K(base_lsn), KPC(palf_handle_impl));
 | |
|     } else if (OB_FAIL(palf_handle_impl->get_min_block_info_for_gc(min_block_id, min_block_max_scn))
 | |
|                && !need_skip_by_ret(ret)) {
 | |
|       ret_code_ = ret;
 | |
|       bool_ret = false;
 | |
|       PALF_LOG(WARN, "LogGetRecycableFileCandidate get_min_block_info_for_gc failed", K(ret), K(palf_id));
 | |
|       // recycable conditions:
 | |
|       // 1. current palf_handle_impl must have some block can be recycable;
 | |
|       // 2. current palf_handle_impl must have older blocks(at least two blocks).
 | |
|       // Always keep there are at least two blocks in range [begin_lsn, base_lsn], because for restart, we will read
 | |
|       // first uncommitted log before base_lsn.
 | |
|     } else if (need_skip_by_ret(ret)
 | |
|                || min_using_block_id < min_block_id
 | |
|                || min_using_block_id - min_block_id < 2) {
 | |
|       PALF_LOG(TRACE, "can not recycle blocks, need keep at least two blocks or has been concurrently"
 | |
|           " with rebuild, skip it",
 | |
|           K(ret), KPC(palf_handle_impl), K(min_block_id), K(min_using_block_id));
 | |
|     } else if (min_block_max_scn_.is_valid() && min_block_max_scn_ < min_block_max_scn) {
 | |
|       PALF_LOG(TRACE, "current palf_handle_impl is not older than previous, skip it", K(min_block_max_scn),
 | |
|           K(min_block_max_scn_), KPC(palf_handle_impl), K(min_block_id));
 | |
|     } else {
 | |
|       id_ = palf_id.id_;
 | |
|       min_block_id_ = min_block_id;
 | |
|       min_block_max_scn_ = min_block_max_scn;
 | |
|       min_using_block_id_ = min_using_block_id;
 | |
|       PALF_LOG(TRACE, "can be recycable palf_handle_impl", K(id_), K(min_block_id_), K(min_using_block_id_),
 | |
|           K(min_block_max_scn_), K(base_lsn));
 | |
|     }
 | |
|     if (min_block_max_scn.is_valid() && (!oldest_block_scn_.is_valid() || oldest_block_scn_ > min_block_max_scn)) {
 | |
|       oldest_block_scn_ = min_block_max_scn;
 | |
|       oldest_palf_id_ = palf_id.id_;
 | |
|     }
 | |
|   }
 | |
|   return bool_ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::try_recycle_blocks()
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   PalfDiskOptions disk_opts_for_stopping_writing;
 | |
|   PalfDiskOptions disk_opts_for_recycling_blocks;
 | |
|   PalfDiskOptionsWrapper::Status status = PalfDiskOptionsWrapper::Status::INVALID_STATUS;
 | |
|   disk_options_wrapper_.get_disk_opts(disk_opts_for_stopping_writing,
 | |
|                                       disk_opts_for_recycling_blocks,
 | |
|                                       status);
 | |
|   int64_t total_used_size_byte = 0;
 | |
|   int64_t total_unrecyclable_size_byte = 0;
 | |
|   int64_t total_size_to_recycle_blocks = disk_opts_for_recycling_blocks.log_disk_usage_limit_size_;
 | |
|   int64_t total_size_to_stop_write = disk_opts_for_stopping_writing.log_disk_usage_limit_size_;
 | |
|   int64_t palf_id = 0;
 | |
|   int64_t maximum_used_size = 0;
 | |
|   int tmp_ret = OB_SUCCESS;
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else if (OB_FAIL(get_disk_usage_(total_used_size_byte, total_unrecyclable_size_byte,
 | |
|                                      palf_id, maximum_used_size))) {
 | |
|     PALF_LOG(WARN, "get_disk_usage_ failed", K(ret), KPC(this));
 | |
|   } else if (FALSE_IT(disk_options_wrapper_.set_cur_unrecyclable_log_disk_size(total_unrecyclable_size_byte))) {
 | |
|   } else if (OB_SUCCESS != (tmp_ret = log_io_worker_wrapper_.notify_need_writing_throttling(disk_options_wrapper_.need_throttling()))) {
 | |
|     PALF_LOG_RET(WARN, tmp_ret, "failed to update_disk_info", K(disk_options_wrapper_));
 | |
|   } else {
 | |
|     const int64_t usable_disk_size_to_recycle_blocks =
 | |
|         total_size_to_recycle_blocks
 | |
|         * disk_opts_for_recycling_blocks.log_disk_utilization_threshold_ / 100LL;
 | |
|     const int64_t usable_disk_limit_size_to_stop_writing =
 | |
|         total_size_to_stop_write
 | |
|         * disk_opts_for_stopping_writing.log_disk_utilization_limit_threshold_ / 100LL;
 | |
|     const bool need_recycle =
 | |
|         usable_disk_size_to_recycle_blocks > total_used_size_byte ? false : true;
 | |
|     const bool is_shrinking = disk_options_wrapper_.is_shrinking();
 | |
|     // Assume that, recycle speed is higher than write speed, therefor, the abnormal case
 | |
|     // is that, after each 'recycle_blocks_', the 'total_used_size_byte' is one PALF_BLOCK_SIZE
 | |
|     // more than 'usable_disk_size'.
 | |
|     const bool curr_diskspace_enough =
 | |
|         usable_disk_limit_size_to_stop_writing > total_used_size_byte ? true : false;
 | |
|     constexpr int64_t MB = 1024 * 1024LL;
 | |
| 
 | |
|     // step1. change SHRINKING_STATUS to normal
 | |
|     // 1. when there is no possibility to stop writing,
 | |
|     // 2. the snapshot of status is SHRINKING_STATUS.
 | |
|     bool has_recycled = false;
 | |
|     int64_t oldest_palf_id = INVALID_PALF_ID;
 | |
|     if (OB_SUCC(ret) && PalfDiskOptionsWrapper::Status::SHRINKING_STATUS == status) {
 | |
|       if (total_used_size_byte < usable_disk_size_to_recycle_blocks) {
 | |
|         disk_options_wrapper_.change_to_normal();
 | |
|         PALF_LOG(INFO, "change_to_normal success", K(disk_options_wrapper_),
 | |
|                  K(total_used_size_byte), K(usable_disk_size_to_recycle_blocks));
 | |
|       }
 | |
|     }
 | |
| 
 | |
|     SCN oldest_scn;
 | |
|     // step2. try recycle blocks
 | |
|     if (true == need_recycle) {
 | |
|       if (OB_FAIL(recycle_blocks_(has_recycled, oldest_palf_id, oldest_scn))) {
 | |
|         PALF_LOG(WARN, "recycle_blocks_ failed", K(usable_disk_size_to_recycle_blocks),
 | |
|                  K(total_used_size_byte), KPC(this));
 | |
|       }
 | |
|     }
 | |
| 
 | |
|     // step3. reset diskspace_enough_.
 | |
|     if (diskspace_enough_ != curr_diskspace_enough) {
 | |
|       ATOMIC_STORE(&diskspace_enough_, curr_diskspace_enough);
 | |
|     }
 | |
|     if ((true == need_recycle && false == has_recycled && false == is_shrinking) || false == diskspace_enough_) {
 | |
|       int tmp_ret = OB_LOG_OUTOF_DISK_SPACE;
 | |
|       LOG_DBA_ERROR(OB_LOG_OUTOF_DISK_SPACE, "msg", "log disk space is almost full", "ret", tmp_ret,
 | |
|           "total_size(MB)", disk_opts_for_recycling_blocks.log_disk_usage_limit_size_/MB,
 | |
|           "used_size(MB)", total_used_size_byte/MB,
 | |
|           "used_percent(%)", (total_used_size_byte* 100) / (disk_opts_for_recycling_blocks.log_disk_usage_limit_size_ + 1),
 | |
|           "warn_size(MB)", (total_size_to_recycle_blocks*disk_opts_for_recycling_blocks.log_disk_utilization_threshold_)/100/MB,
 | |
|           "warn_percent(%)", disk_opts_for_recycling_blocks.log_disk_utilization_threshold_,
 | |
|           "limit_size(MB)", (total_size_to_recycle_blocks*disk_opts_for_recycling_blocks.log_disk_utilization_limit_threshold_)/100/MB,
 | |
|           "limit_percent(%)", disk_opts_for_recycling_blocks.log_disk_utilization_limit_threshold_,
 | |
|           "total_unrecyclable_size_byte(MB)", total_unrecyclable_size_byte/MB,
 | |
|           "maximum_used_size(MB)", maximum_used_size/MB,
 | |
|           "maximum_log_stream", palf_id,
 | |
|           "oldest_log_stream", oldest_palf_id,
 | |
|           "oldest_scn", oldest_scn);
 | |
|     } else {
 | |
|        if (REACH_TIME_INTERVAL(2 * 1000 * 1000L)) {
 | |
|          PALF_LOG(INFO, "LOG_DISK_OPTION", K(disk_options_wrapper_));
 | |
|        }
 | |
|     }
 | |
| 
 | |
|     (void)remove_stale_incomplete_palf_();
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| bool PalfEnvImpl::check_disk_space_enough()
 | |
| {
 | |
|   return true == ATOMIC_LOAD(&diskspace_enough_);
 | |
| }
 | |
| 
 | |
| PalfEnvImpl::GetTotalUsedDiskSpace::GetTotalUsedDiskSpace()
 | |
|   : total_used_disk_space_(0), total_unrecyclable_disk_space_(0), maximum_used_size_(0), palf_id_(INVALID_PALF_ID) {}
 | |
| PalfEnvImpl::GetTotalUsedDiskSpace::~GetTotalUsedDiskSpace() {}
 | |
| 
 | |
| bool PalfEnvImpl::GetTotalUsedDiskSpace::operator() (const LSKey &ls_key, IPalfHandleImpl *palf_handle_impl)
 | |
| {
 | |
|   bool bool_ret = true;
 | |
|   if (NULL == palf_handle_impl) {
 | |
|     ret_code_ = OB_ERR_UNEXPECTED;
 | |
|     bool_ret = false;
 | |
|   } else {
 | |
|     constexpr int64_t MB = 1024 * 1024;
 | |
|     int ret = OB_SUCCESS;
 | |
|     int64_t used_size = 0;
 | |
|     int64_t unrecyclable_size = 0;
 | |
|     if (OB_FAIL(palf_handle_impl->get_total_used_disk_space(used_size, unrecyclable_size))) {
 | |
|       PALF_LOG(WARN, "failed to get_total_used_disk_space", K(ls_key));
 | |
|       ret_code_ = ret;
 | |
|       bool_ret = false;
 | |
|     } else {
 | |
|       if (used_size >= maximum_used_size_) {
 | |
|         maximum_used_size_ = used_size;
 | |
|         palf_id_ = ls_key.id_;
 | |
|       }
 | |
|       total_used_disk_space_ += used_size;
 | |
|       total_unrecyclable_disk_space_ += unrecyclable_size;
 | |
|       PALF_LOG(TRACE, "get_total_used_disk_space success", K(ls_key),
 | |
|              "total_used_disk_space(MB):", total_used_disk_space_/MB,
 | |
|              "total_unrecyclable_disk_space(MB):", total_unrecyclable_disk_space_/MB,
 | |
|              "end_lsn", palf_handle_impl->get_end_lsn());
 | |
|     }
 | |
|   }
 | |
|   return bool_ret;
 | |
| }
 | |
| 
 | |
| PalfEnvImpl::RemoveStaleIncompletePalfFunctor::RemoveStaleIncompletePalfFunctor(PalfEnvImpl *palf_env_impl)
 | |
|   : palf_env_impl_(palf_env_impl)
 | |
| {}
 | |
| 
 | |
|  PalfEnvImpl::RemoveStaleIncompletePalfFunctor::~RemoveStaleIncompletePalfFunctor()
 | |
| {
 | |
|   palf_env_impl_ = NULL;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::RemoveStaleIncompletePalfFunctor::func(const dirent *entry)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   char file_name[OB_MAX_FILE_NAME_LENGTH] = {'\0'};
 | |
|   const char *d_name = entry->d_name;
 | |
|   MEMCPY(file_name, d_name, strlen(d_name));
 | |
|   char *tmp = strtok(file_name, "_");
 | |
|   char *timestamp_str = NULL;
 | |
|   if (NULL == tmp || NULL == (timestamp_str = strtok(NULL, "_"))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(WARN, "unexpected format", K(ret), K(tmp), K(file_name));
 | |
|   } else {
 | |
|     int64_t timestamp = atol(timestamp_str);
 | |
|     int64_t current_timestamp = ObTimeUtility::current_time();
 | |
|     int64_t delta = current_timestamp - timestamp;
 | |
|     constexpr int64_t week_us = 7 * 24 * 60 * 60 * 1000 * 1000ll;
 | |
|     if (delta <= week_us) {
 | |
|       PALF_LOG(TRACE, "no need remove this incomplet dir", K(d_name), K(delta),
 | |
|           K(timestamp), K(timestamp_str), K(current_timestamp));
 | |
|     } else {
 | |
|       char path[OB_MAX_FILE_NAME_LENGTH] = {'\0'};
 | |
|       int pret = OB_SUCCESS;
 | |
|       if (0 > (pret = snprintf(path, MAX_PATH_SIZE, "%s/%s", palf_env_impl_->tmp_log_dir_, d_name))) {
 | |
|         ret = OB_ERR_UNEXPECTED;
 | |
|         PALF_LOG(WARN, "snprintf failed", K(ret), K(file_name), K(d_name));
 | |
|       } else if (OB_FAIL(FileDirectoryUtils::delete_directory_rec(path))) {
 | |
|         PALF_LOG(WARN, "delete_directory_rec failed", K(ret), K(file_name), K(path), K(entry->d_name), K(timestamp_str));
 | |
|       } else {
 | |
|         PALF_LOG(WARN, "current incomplete palf has bee staled, delete it", K(timestamp), K(current_timestamp), K(path));
 | |
|       }
 | |
|     }
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_disk_usage(int64_t &used_size_byte, int64_t &total_usable_size_byte)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   constexpr int64_t MB = 1024 * 1024;
 | |
|   PalfDiskOptions disk_options = disk_options_wrapper_.get_disk_opts_for_recycling_blocks();
 | |
|   if (OB_FAIL(get_disk_usage_(used_size_byte))) {
 | |
|     PALF_LOG(WARN, "get_disk_usage_ failed", K(ret));
 | |
|   } else {
 | |
|     total_usable_size_byte = disk_options.log_disk_usage_limit_size_;
 | |
|     PALF_LOG(INFO, "get_disk_usage", K(ret), "capacity(MB):", total_usable_size_byte/MB, "used(MB):", used_size_byte/MB);
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::update_options(const PalfOptions &options)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   WLockGuard guard(palf_meta_lock_);
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else if (false == options.is_valid()) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     PALF_LOG(WARN, "invalid argument", K(options));
 | |
|   } else if (OB_FAIL(disk_options_wrapper_.update_disk_options(options.disk_options_))) {
 | |
|     PALF_LOG(WARN, "update_disk_options failed", K(ret), K(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));
 | |
|   } else {
 | |
|     rebuild_replica_log_lag_threshold_ = options.rebuild_replica_log_lag_threshold_;
 | |
|     PALF_LOG(INFO, "update_palf_options success", K(options));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_options(PalfOptions &options)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else {
 | |
|     options.disk_options_ = disk_options_wrapper_.get_disk_opts_for_recycling_blocks();
 | |
|     options.compress_options_ = log_rpc_.get_compress_opts();
 | |
|     options.rebuild_replica_log_lag_threshold_ = rebuild_replica_log_lag_threshold_;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::for_each(const common::ObFunction<int (IPalfHandleImpl *)> &func)
 | |
| {
 | |
|   auto func_impl = [&func](const LSKey &ls_key, IPalfHandleImpl *ipalf_handle_impl) -> bool {
 | |
|     bool bool_ret = true;
 | |
|     int ret = OB_SUCCESS;
 | |
|     if (OB_FAIL(func(ipalf_handle_impl))) {
 | |
|       PALF_LOG(WARN, "execute func failed", K(ret), K(ls_key));
 | |
|     }
 | |
|     if (OB_FAIL(ret)) {
 | |
|      bool_ret = false;
 | |
|     }
 | |
|     return bool_ret;
 | |
|   };
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_FAIL(palf_handle_impl_map_.for_each(func_impl))) {
 | |
|     PALF_LOG(WARN, "iterate palf_handle_impl_map_ failed", K(ret));
 | |
|   } else {
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::for_each(const common::ObFunction<int (const PalfHandle &)> &func)
 | |
| {
 | |
|   auto func_impl = [&func](const LSKey &ls_key, IPalfHandleImpl *ipalf_handle_impl) -> bool {
 | |
|     bool bool_ret = true;
 | |
|     int ret = OB_SUCCESS;
 | |
|     PalfHandle palf_handle;
 | |
|     palf_handle.palf_handle_impl_ = ipalf_handle_impl;
 | |
|     if (OB_FAIL(func(palf_handle))) {
 | |
|       PALF_LOG(WARN, "execute func failed", K(ret), K(ls_key));
 | |
|     } else {
 | |
|     }
 | |
|     if (OB_FAIL(ret)) {
 | |
|      bool_ret = false;
 | |
|     }
 | |
|     return bool_ret;
 | |
|   };
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_FAIL(palf_handle_impl_map_.for_each(func_impl))) {
 | |
|     PALF_LOG(WARN, "iterate palf_handle_impl_map_ failed", K(ret));
 | |
|   } else {
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| common::ObILogAllocator* PalfEnvImpl::get_log_allocator()
 | |
| {
 | |
|   return log_alloc_mgr_;
 | |
| }
 | |
| 
 | |
| PalfEnvImpl::ReloadPalfHandleImplFunctor::ReloadPalfHandleImplFunctor(PalfEnvImpl *palf_env_impl) : palf_env_impl_(palf_env_impl)
 | |
| {
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::ReloadPalfHandleImplFunctor::func(const struct dirent *entry)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   ObTimeGuard guard("ReloadFunctor");
 | |
|   struct stat st;
 | |
|   char log_dir[OB_MAX_FILE_NAME_LENGTH] = {'\0'};
 | |
|   if (OB_ISNULL(entry)) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     PALF_LOG(WARN, "invalid args", K(ret), KP(entry));
 | |
|   } else if (0 > (pret = snprintf(log_dir, MAX_PATH_SIZE, "%s/%s", palf_env_impl_->log_dir_, entry->d_name))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(WARN, "snprint failed", K(ret), K(pret), K(entry->d_name));
 | |
|   } else if (0 != stat(log_dir, &st)) {
 | |
|     PALF_LOG(INFO, "this entry is not a block", K(ret), K(log_dir), K(errno));
 | |
|   } else if (false == S_ISDIR(st.st_mode)) {
 | |
|     PALF_LOG(WARN, "path is not a directory, ignore it", K(ret), K(log_dir), K(st.st_mode));
 | |
|   } else {
 | |
|     const char *path = entry->d_name;
 | |
|     bool is_number = true;
 | |
|     const size_t path_len = strlen(path);
 | |
|     for (size_t i = 0; is_number && i < path_len; ++i) {
 | |
|       if ('\0' == path[i]) {
 | |
|         break;
 | |
|       } else if (!isdigit(path[i])) {
 | |
|         is_number = false;
 | |
|       }
 | |
|     }
 | |
|     if (!is_number) {
 | |
|       // do nothing, skip invalid block like tmp
 | |
|     } else {
 | |
|       int64_t id = strtol(path, nullptr, 10);
 | |
|       if (OB_FAIL(palf_env_impl_->reload_palf_handle_impl_(id))) {
 | |
|         PALF_LOG(WARN, "reload_palf_handle_impl failed", K(ret));
 | |
|       }
 | |
|       guard.click("reload_palf_handle_impl");
 | |
|       PALF_LOG(INFO, "reload_palf_handle_impl_", K(ret), K(id), K(guard));
 | |
|     }
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::reload_palf_handle_impl_(const int64_t palf_id)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   PalfHandleImpl *tmp_palf_handle_impl;
 | |
|   char base_dir[OB_MAX_FILE_NAME_LENGTH] = {'\0'};
 | |
|   int64_t start_ts = ObTimeUtility::current_time();
 | |
|   LSKey hash_map_key(palf_id);
 | |
|   bool is_integrity = true;
 | |
|   const int64_t palf_epoch = ATOMIC_AAF(&last_palf_epoch_, 1);
 | |
|   if (0 > (pret = snprintf(base_dir, MAX_PATH_SIZE, "%s/%ld", log_dir_, palf_id))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(WARN, "snprint failed", K(ret), K(pret), K(palf_id));
 | |
|   } else if (NULL == (tmp_palf_handle_impl = PalfHandleImplFactory::alloc())) {
 | |
|     ret = OB_ALLOCATE_MEMORY_FAILED;
 | |
|     PALF_LOG(WARN, "alloc ipalf_handle_impl failed", K(ret));
 | |
|   } else if (OB_FAIL(tmp_palf_handle_impl->load(palf_id, &fetch_log_engine_, base_dir, log_alloc_mgr_,
 | |
|           log_block_pool_, &log_rpc_, log_io_worker_wrapper_.get_log_io_worker(palf_id), this, self_,
 | |
|           &election_timer_, palf_epoch, is_integrity))) {
 | |
|     PALF_LOG(ERROR, "PalfHandleImpl init failed", K(ret), K(palf_id));
 | |
|   } else if (OB_FAIL(palf_handle_impl_map_.insert_and_get(hash_map_key, tmp_palf_handle_impl))) {
 | |
|     PALF_LOG(WARN, "palf_handle_impl_map_ insert_and_get failed", K(ret), K(palf_id), K(tmp_palf_handle_impl));
 | |
|   } else {
 | |
|     (void) tmp_palf_handle_impl->set_monitor_cb(monitor_);
 | |
|     (void) tmp_palf_handle_impl->set_scan_disk_log_finished();
 | |
|     palf_handle_impl_map_.revert(tmp_palf_handle_impl);
 | |
|     int64_t cost_ts = ObTimeUtility::current_time() - start_ts;
 | |
|     PALF_LOG(INFO, "reload_palf_handle_impl success", K(ret), K(palf_id), K(cost_ts), KP(this),
 | |
|         KP(&palf_handle_impl_map_));
 | |
|   }
 | |
| 
 | |
|   if (OB_FAIL(ret) && NULL != tmp_palf_handle_impl) {
 | |
|     // if 'tmp_palf_handle_impl' has not been inserted into hash map,
 | |
|     // need reclaim manually.
 | |
|     PALF_LOG(ERROR, "reload_palf_handle_impl_ failed, need free tmp_palf_handle_impl", K(ret), K(tmp_palf_handle_impl));
 | |
|     if (OB_ENTRY_NOT_EXIST == palf_handle_impl_map_.contains_key(hash_map_key)) {
 | |
|       PalfHandleImplFactory::free(tmp_palf_handle_impl);
 | |
|       tmp_palf_handle_impl = NULL;
 | |
|     }
 | |
|   } else if (false == is_integrity) {
 | |
|     PALF_LOG(WARN, "palf instance is not integrity, remove it", K(palf_id));
 | |
|     ret = move_incomplete_palf_into_tmp_dir_(palf_id);
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_total_used_disk_space_(int64_t &total_used_disk_space,
 | |
|                                             int64_t &total_unrecyclable_disk_space,
 | |
|                                             int64_t &palf_id,
 | |
|                                             int64_t &maximum_used_size)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   GetTotalUsedDiskSpace functor;
 | |
|   if (OB_FAIL(palf_handle_impl_map_.for_each(functor))) {
 | |
|     ret = functor.ret_code_;
 | |
|     PALF_LOG(WARN, "get_total_used_disk_space", K(ret), K(functor));
 | |
|   } else {
 | |
|     palf_id = functor.palf_id_;
 | |
|     maximum_used_size = functor.maximum_used_size_;
 | |
|     total_used_disk_space = functor.total_used_disk_space_;
 | |
|     total_unrecyclable_disk_space = functor.total_unrecyclable_disk_space_;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_disk_usage_(int64_t &used_size_byte,
 | |
|                                  int64_t &unrecyclable_disk_space,
 | |
|                                  int64_t &palf_id,
 | |
|                                  int64_t &maximum_used_size)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_FAIL(get_total_used_disk_space_(used_size_byte, unrecyclable_disk_space, palf_id, maximum_used_size))) {
 | |
|     PALF_LOG(WARN, "get_total_used_disk_space failed", K(ret), KPC(this));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_disk_usage_(int64_t &used_size_byte)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int64_t unused_unrecyclable_size = 0;
 | |
|   int64_t unused_palf_id = 0;
 | |
|   int64_t unused_size = 0;
 | |
|   if (OB_FAIL(get_disk_usage_(used_size_byte, unused_unrecyclable_size, unused_palf_id, unused_size))) {
 | |
|     PALF_LOG(WARN, "get_total_used_disk_space failed", K(ret), KPC(this));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::recycle_blocks_(bool &has_recycled, int64_t &oldest_palf_id, SCN &oldest_scn)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   has_recycled = false;
 | |
|   // TODO by runlin:
 | |
|   //  1. only execute unlink blocks when the disk usage watemark reaches the threshold.
 | |
|   //  2. batch unlink in each round.
 | |
|   LogGetRecycableFileCandidate functor;
 | |
|   if (OB_FAIL(palf_handle_impl_map_.for_each(functor))) {
 | |
|     PALF_LOG(WARN, "palf_handle_impl_map_ for_each failed", K(ret), K(functor));
 | |
|   } else {
 | |
|     IPalfHandleImplGuard guard;
 | |
|     int64_t palf_id(functor.id_);
 | |
|     const block_id_t min_block_id = functor.min_block_id_;
 | |
|     if (false == is_valid_block_id(min_block_id)) {
 | |
|       PALF_LOG(WARN, "there is not any block can be recycled, need verify the base"
 | |
|           "lsn of PalfHandleImpl whether has been advanced", K(ret), KPC(this));
 | |
|     } else if (OB_FAIL(get_palf_handle_impl(palf_id, guard))) {
 | |
|       PALF_LOG(WARN, "get_palf_handle_impl failed", K(ret), K(palf_id));
 | |
|     } else if (OB_FAIL(guard.get_palf_handle_impl()->delete_block(min_block_id))) {
 | |
|       PALF_LOG(WARN, "delete block failed", K(ret), K(min_block_id), K(functor));
 | |
|     } else {
 | |
|       has_recycled = true;
 | |
|       PALF_LOG(INFO, "recycle_blocks success", K(functor));
 | |
|     }
 | |
|     oldest_palf_id = functor.oldest_palf_id_;
 | |
|     oldest_scn = functor.oldest_block_scn_;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::wait_until_reference_count_to_zero_(const int64_t palf_id)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   char base_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   char tmp_base_dir[MAX_PATH_SIZE] = {'\0'};
 | |
|   if (false == is_valid_palf_id(palf_id)) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     PALF_LOG(ERROR, "invalid arguments", K(ret), K(palf_id));
 | |
|   } else if (0 > (pret = snprintf(base_dir, MAX_PATH_SIZE, "%s/%ld", log_dir_, palf_id))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprinf failed", K(ret), K(pret), K(palf_id));
 | |
|   } else if (0 > (pret = snprintf(tmp_base_dir, MAX_PATH_SIZE, "%s/%ld%s", log_dir_, palf_id, TMP_SUFFIX))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprinf failed", K(ret), K(pret), K(palf_id));
 | |
|   } else {
 | |
|     bool normal_dir_exist = true;
 | |
|     bool tmp_dir_exist = true;
 | |
|     while (OB_SUCC(FileDirectoryUtils::is_exists(base_dir, normal_dir_exist))
 | |
|            && OB_SUCC(FileDirectoryUtils::is_exists(tmp_base_dir, tmp_dir_exist))) {
 | |
|       if (!normal_dir_exist && !tmp_dir_exist) {
 | |
|         break;
 | |
|       }
 | |
|       PALF_LOG(INFO, "wait_until_reference_count_to_zero_ failed, may be reference count has leaked", K(palf_id),
 | |
|           K(normal_dir_exist), K(tmp_dir_exist), K(base_dir), K(tmp_base_dir));
 | |
|       ob_usleep(1000);
 | |
|     }
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| bool PalfEnvImpl::check_can_create_palf_handle_impl_() const
 | |
| {
 | |
|   bool bool_ret = true;
 | |
|   int64_t count = palf_handle_impl_map_.count();
 | |
|   // NB: avoid concurrent with expand and shrink, need guard by palf_meta_lock_.
 | |
|   const PalfDiskOptions disk_opts = disk_options_wrapper_.get_disk_opts_for_recycling_blocks();
 | |
|   bool_ret = (count + 1) * MIN_DISK_SIZE_PER_PALF_INSTANCE <= disk_opts.log_disk_usage_limit_size_;
 | |
|   return bool_ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::remove_palf_handle_impl_from_map_not_guarded_by_lock_(const int64_t palf_id)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   LSKey hash_map_key(palf_id);
 | |
|   auto set_delete_func = [](const LSKey &key, IPalfHandleImpl *value) {
 | |
|     UNUSED(key);
 | |
|     value->set_deleted();
 | |
|   };
 | |
|   if (OB_FAIL(palf_handle_impl_map_.operate(hash_map_key, set_delete_func))) {
 | |
|     PALF_LOG(WARN, "operate palf_handle_impl_map_ failed", K(ret), K(palf_id), KPC(this));
 | |
|   } else if (OB_FAIL(palf_handle_impl_map_.del(hash_map_key))) {
 | |
|     PALF_LOG(WARN, "palf_handle_impl_map_ del failed", K(ret), K(palf_id));
 | |
|   } else {
 | |
|     PALF_LOG(INFO, "remove_palf_handle_impl success", K(ret), K(palf_id));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::move_incomplete_palf_into_tmp_dir_(const int64_t palf_id)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   int pret = 0;
 | |
|   const mode_t mode = S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH;
 | |
|   char src_log_dir[OB_MAX_FILE_NAME_LENGTH] = {'\0'};
 | |
|   char dest_log_dir[OB_MAX_FILE_NAME_LENGTH] = {'\0'};
 | |
|   bool tmp_dir_exist = false;
 | |
|   LSKey hash_map_key(palf_id);
 | |
|   int64_t timestamp = ObTimeUtility::current_time();
 | |
|   if (OB_FAIL(palf_handle_impl_map_.del(hash_map_key))) {
 | |
|     PALF_LOG(WARN, "del palf from map failed, unexpected", K(ret),
 | |
|         K(palf_id), KPC(this));;
 | |
|   } else if (OB_FAIL(check_tmp_log_dir_exist_(tmp_dir_exist))) {
 | |
|   } else if (false == tmp_dir_exist && (-1 == ::mkdir(tmp_log_dir_, mode))) {
 | |
|     ret = convert_sys_errno();
 | |
|     PALF_LOG(ERROR, "mkdir tmp log dir failed", K(ret), KPC(this), K(tmp_log_dir_));
 | |
|   } else if (0 > (pret = snprintf(src_log_dir, MAX_PATH_SIZE, "%s/%ld", log_dir_, palf_id))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprintf failed, unexpected error", K(ret));
 | |
|   } else if (0 > (pret = snprintf(dest_log_dir, MAX_PATH_SIZE, "%s/%ld_%ld", tmp_log_dir_, palf_id, timestamp))) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     PALF_LOG(ERROR, "snprintf failed, unexpected error", K(ret));
 | |
|   } else if (-1 == ::rename(src_log_dir, dest_log_dir)) {
 | |
|     ret = convert_sys_errno();
 | |
|     PALF_LOG(ERROR, "::rename failed", K(ret), KPC(this), K(src_log_dir), K(dest_log_dir));
 | |
|   } else if (OB_FAIL(FileDirectoryUtils::fsync_dir(log_dir_))) {
 | |
|     PALF_LOG(ERROR, "fsync_dir failed", K(ret), KPC(this), K(src_log_dir), K(dest_log_dir));
 | |
|   } else {
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::check_tmp_log_dir_exist_(bool &exist) const
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_FAIL(FileDirectoryUtils::is_exists(tmp_log_dir_, exist))) {
 | |
|     PALF_LOG(WARN, "check dir exist failed", K(ret), KPC(this), K(tmp_log_dir_));
 | |
|   } else {
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::remove_stale_incomplete_palf_()
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   bool exist = false;
 | |
|   RemoveStaleIncompletePalfFunctor functor(this);
 | |
|   if (OB_FAIL(check_tmp_log_dir_exist_(exist))) {
 | |
|     PALF_LOG(WARN, "check_tmp_log_dir_exist_ failed", K(ret), KPC(this));
 | |
|   } else if (false == exist) {
 | |
|   } else if (OB_FAIL(scan_dir(tmp_log_dir_, functor))){
 | |
|     PALF_LOG(WARN, "remove_stale_incomplete_palf_ failed", K(ret), KPC(this), K(tmp_log_dir_));
 | |
|   } else {
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_io_start_time(int64_t &last_working_time)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else {
 | |
|     last_working_time = log_io_worker_wrapper_.get_last_working_time();
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int64_t PalfEnvImpl::get_tenant_id()
 | |
| {
 | |
|   return tenant_id_;
 | |
| }
 | |
| int PalfEnvImpl::update_replayable_point(const SCN &replayable_scn)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else if (OB_FAIL(fetch_log_engine_.update_replayable_point(replayable_scn))) {
 | |
|     PALF_LOG(WARN, "update_replayable_point failed", KPC(this), K(replayable_scn));
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int PalfEnvImpl::get_throttling_options(PalfThrottleOptions &options)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (IS_NOT_INIT) {
 | |
|     ret = OB_NOT_INIT;
 | |
|   } else {
 | |
|     (void)disk_options_wrapper_.get_throttling_options(options);
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| } // end namespace palf
 | |
| } // end namespace oceanbase
 | 
