/** * Copyright (c) 2021 OceanBase * OceanBase CE is licensed under Mulan PubL v2. * You can use this software according to the terms and conditions of the Mulan PubL v2. * You may obtain a copy of Mulan PubL v2 at: * http://license.coscl.org.cn/MulanPubL-2.0 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. * See the Mulan PubL v2 for more details. */ #define USING_LOG_PREFIX SQL_ENG #include "ob_tenant_sql_memory_manager.h" #include "share/rc/ob_tenant_base.h" #include "share/inner_table/ob_inner_table_schema_constants.h" #include "share/system_variable/ob_system_variable_alias.h" #include "share/schema/ob_schema_getter_guard.h" #include "observer/omt/ob_tenant_config_mgr.h" #include "lib/alloc/alloc_func.h" #include "lib/rc/ob_rc.h" #include "observer/ob_server_struct.h" #include "sql/engine/px/ob_px_util.h" namespace oceanbase { using namespace lib; using namespace omt; using namespace share; using namespace common; using namespace share::schema; using namespace oceanbase::observer; namespace sql { //////////////////////////////////////////////////////////////////////////////////// const int64_t ObSqlWorkAreaProfile::MIN_BOUND_SIZE[ObSqlWorkAreaType::MAX_TYPE] = { 9 * OB_MALLOC_MIDDLE_BLOCK_SIZE, // HASH OB_MALLOC_MIDDLE_BLOCK_SIZE, // SORT }; int64_t ObSqlWorkAreaProfile::get_dop() { int64_t dop = 1; if (OB_NOT_NULL(exec_ctx_)) { dop = ObPxSqcUtil::get_actual_worker_count(exec_ctx_); } return dop; } uint64_t ObSqlWorkAreaProfile::get_plan_id() { uint64_t plan_id = UINT64_MAX; if (OB_NOT_NULL(exec_ctx_)) { plan_id = ObPxSqcUtil::get_plan_id(exec_ctx_); } return plan_id; } uint64_t ObSqlWorkAreaProfile::get_exec_id() { uint64_t exec_id = UINT64_MAX; if (OB_NOT_NULL(exec_ctx_)) { exec_id = ObPxSqcUtil::get_exec_id(exec_ctx_); } return exec_id; } const char* ObSqlWorkAreaProfile::get_sql_id() { const char* sql_id = nullptr; if (OB_NOT_NULL(exec_ctx_)) { sql_id = ObPxSqcUtil::get_sql_id(exec_ctx_); } return sql_id; } uint64_t ObSqlWorkAreaProfile::get_session_id() { uint64_t session_id = UINT64_MAX; if (OB_NOT_NULL(exec_ctx_)) { session_id = ObPxSqcUtil::get_session_id(exec_ctx_); } return session_id; } //////////////////////////////////////////////////////////////////////////////////// int ObSqlWorkAreaIntervalStat::analyze_profile(ObSqlWorkAreaProfile& profile, int64_t cache_size, const int64_t one_pass_size, const int64_t max_size, bool is_one_pass) { int ret = OB_SUCCESS; if (is_one_pass) { if (profile.is_sort_wa()) { ++total_one_pass_cnt_; total_one_pass_size_ += cache_size; } } else { if (max_size <= cache_size) { cache_size = max_size; profile.init(max_size, profile.get_chunk_size()); } if (profile.is_hash_join_wa()) { ++total_hash_cnt_; total_hash_size_ += cache_size; } else if (profile.is_sort_wa()) { ++total_sort_cnt_; total_sort_size_ += cache_size; total_sort_one_pass_size_ += one_pass_size; } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpect profile type", K(ret), K(profile.get_work_area_type())); } } return ret; } void ObSqlWorkAreaIntervalStat::reset() { total_hash_cnt_ = 0; total_hash_size_ = 0; total_sort_cnt_ = 0; total_sort_size_ = 0; total_sort_one_pass_size_ = 0; total_one_pass_cnt_ = 0; total_one_pass_size_ = 0; } //////////////////////////////////////////////////////////////////////////////////// void ObSqlMemoryList::reset() { ObLockGuard lock_guard(lock_); DLIST_FOREACH_REMOVESAFE_NORET(profile, profile_list_) { profile_list_.remove(profile); profile->set_expect_size(OB_INVALID_ID); } profile_list_.reset(); } int ObSqlMemoryList::register_work_area_profile(ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; ObLockGuard lock_guard(lock_); profile_list_.add_last(&profile); return ret; } int ObSqlMemoryList::unregister_work_area_profile(ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; ObLockGuard lock_guard(lock_); profile_list_.remove(&profile); return ret; } //////////////////////////////////////////////////////////////////////////////////// int ObTenantSqlMemoryManager::ObSqlWorkAreaCalcInfo::init( ObIAllocator& allocator, ObSqlWorkAreaInterval* wa_intervals, int64_t interval_cnt) { int ret = OB_SUCCESS; if (OB_ISNULL(wa_intervals)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: work interval is null", K(ret)); } else { wa_intervals_ = reinterpret_cast(allocator.alloc(sizeof(ObSqlWorkAreaInterval) * interval_cnt)); if (nullptr == wa_intervals_) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc work area interval", K(ret)); } else { for (int64_t i = 0; i < interval_cnt; ++i) { void* buf = static_cast(&wa_intervals_[i]); ObSqlWorkAreaInterval* wa_interval = new (buf) ObSqlWorkAreaInterval(i, wa_intervals[i].get_interval_cache_size()); UNUSED(wa_interval); } } } return ret; } void ObTenantSqlMemoryManager::ObSqlWorkAreaCalcInfo::destroy(ObIAllocator& allocator) { if (OB_NOT_NULL(wa_intervals_)) { allocator.free(wa_intervals_); wa_intervals_ = nullptr; } } int ObTenantSqlMemoryManager::ObSqlWorkAreaCalcInfo::calc_memory_target(int64_t idx, const int64_t pre_mem_target) { int ret = OB_SUCCESS; int64_t dst_mem_target = pre_mem_target; if (INTERVAL_NUM - 1 == idx) { wa_intervals_[idx].set_mem_target(dst_mem_target); } else { ObSqlWorkAreaIntervalStat& pre_interval_stat = wa_intervals_[idx + 1].get_interval_stat(); int64_t hash_delta = pre_interval_stat.get_total_hash_size() - wa_intervals_[idx].get_interval_cache_size() * pre_interval_stat.get_total_hash_cnt() + tmp_no_cache_cnt_ * (wa_intervals_[idx + 1].get_interval_cache_size() - wa_intervals_[idx].get_interval_cache_size()); int64_t sort_delta = pre_interval_stat.get_total_sort_size() - pre_interval_stat.get_total_sort_one_pass_size(); int64_t one_pass_delta = pre_interval_stat.get_total_one_pass_size() - wa_intervals_[idx].get_interval_cache_size() * pre_interval_stat.get_total_one_pass_cnt(); dst_mem_target -= hash_delta; dst_mem_target -= sort_delta; dst_mem_target -= one_pass_delta; if (0 > hash_delta || 0 > sort_delta || 0 > dst_mem_target) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected delta size", K(hash_delta), K(sort_delta), K(dst_mem_target), K(idx), K(pre_mem_target), K(pre_interval_stat.get_total_hash_size()), K(wa_intervals_[idx].get_interval_cache_size()), K(pre_interval_stat.get_total_hash_cnt()), K(one_pass_delta), K(pre_interval_stat.get_total_one_pass_size()), K(pre_interval_stat.get_total_one_pass_cnt()), K(tmp_no_cache_cnt_)); } else { wa_intervals_[idx].set_mem_target(dst_mem_target); } if (hash_delta > 0 || sort_delta > 0 || one_pass_delta > 0) { LOG_TRACE("trace memory target", K(hash_delta), K(sort_delta), K(dst_mem_target), K(idx), K(pre_mem_target), K(pre_interval_stat.get_total_hash_size()), K(wa_intervals_[idx].get_interval_cache_size()), K(pre_interval_stat.get_total_hash_cnt()), K(one_pass_delta), K(pre_interval_stat.get_total_one_pass_size()), K(pre_interval_stat.get_total_one_pass_cnt()), K(dst_mem_target), K(tmp_no_cache_cnt_)); } tmp_no_cache_cnt_ += (pre_interval_stat.get_total_hash_cnt() + pre_interval_stat.get_total_one_pass_cnt()); } return ret; } int ObTenantSqlMemoryManager::ObSqlWorkAreaCalcInfo::find_best_interval_index_by_mem_target( int64_t& interval_idx, const int64_t expect_mem_target, const int64_t total_memory_size) { int ret = OB_SUCCESS; int64_t pre_mem_target = total_memory_size; int64_t delta = INT64_MAX; interval_idx = -1; for (int64_t i = INTERVAL_NUM - 1; i >= 0 && OB_SUCC(ret); --i) { if (OB_FAIL(calc_memory_target(i, pre_mem_target))) { LOG_WARN("failed to calculate memory target", K(i), K(pre_mem_target)); } else { int64_t mem_target = wa_intervals_[i].get_mem_target(); if (mem_target <= expect_mem_target && expect_mem_target - mem_target <= delta) { interval_idx = i; delta = expect_mem_target - mem_target; break; } pre_mem_target = mem_target; } } mem_target_ = expect_mem_target; return ret; } int ObTenantSqlMemoryManager::ObSqlWorkAreaCalcInfo::calculate_global_bound_size( const int64_t wa_max_memory_size, const int64_t total_memory_size, const int64_t profile_cnt, const bool auto_calc) { int ret = OB_SUCCESS; int64_t max_wa_size = wa_max_memory_size; int64_t max_bound_size = (max_wa_size >> 3); profile_cnt_ = profile_cnt; int64_t avg_bound_size = (0 == profile_cnt_) ? max_bound_size : max_wa_size / profile_cnt_; int64_t best_interval_idx = -1; if (OB_FAIL(find_best_interval_index_by_mem_target(best_interval_idx, max_wa_size, total_memory_size))) { LOG_WARN("failed to find best interval index", K(ret), K(best_interval_idx), K(max_wa_size), K(total_memory_size)); } else { int64_t calc_global_bound_size = 0; if (-1 == best_interval_idx) { global_bound_size_ = LESS_THAN_100M_INTERVAL_SIZE; } else { calc_global_bound_size = wa_intervals_[best_interval_idx].get_interval_cache_size(); global_bound_size_ = calc_global_bound_size; } if (global_bound_size_ > max_bound_size) { global_bound_size_ = max_bound_size; } if (global_bound_size_ < min_bound_size_) { global_bound_size_ = min_bound_size_; } if (auto_calc) { LOG_INFO("timer to calc global bound size", K(ret), K(best_interval_idx), K(global_bound_size_), K(calc_global_bound_size), K(mem_target_), K(wa_max_memory_size), K(profile_cnt_), K(total_memory_size), K(max_wa_size), K(avg_bound_size), K(max_bound_size), K(min_bound_size_)); } } return ret; } //////////////////////////////////////////////////////////////////////////////////// int ObTenantSqlMemoryManager::mtl_init(ObTenantSqlMemoryManager*& sql_mem_mgr) { int ret = OB_SUCCESS; uint64_t tenant_id = oceanbase::lib::current_tenant_id(); sql_mem_mgr = nullptr; if (OB_MAX_RESERVED_TENANT_ID < tenant_id) { sql_mem_mgr = OB_NEW(ObTenantSqlMemoryManager, common::ObModIds::OB_SQL_EXECUTOR, tenant_id); if (nullptr == sql_mem_mgr) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc tenant sql memory manager", K(ret)); } else if (OB_FAIL( sql_mem_mgr->allocator_.init(lib::ObMallocAllocator::get_instance(), OB_MALLOC_NORMAL_BLOCK_SIZE))) { LOG_WARN("failed to init fifo allocator", K(ret)); } else { sql_mem_mgr->allocator_.set_label(ObModIds::OB_SQL_EXECUTOR); int64_t work_area_interval_size = sizeof(ObSqlWorkAreaInterval) * INTERVAL_NUM; sql_mem_mgr->wa_intervals_ = reinterpret_cast(sql_mem_mgr->allocator_.alloc(work_area_interval_size)); sql_mem_mgr->profile_lists_ = reinterpret_cast(sql_mem_mgr->allocator_.alloc(sizeof(ObSqlMemoryList) * HASH_CNT)); if (nullptr == sql_mem_mgr->wa_intervals_) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc work area interval", K(ret)); } else if (nullptr == sql_mem_mgr->profile_lists_) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc profile list", K(ret)); } else { sql_mem_mgr->tenant_id_ = tenant_id; // 1M int64_t total_size = 0; int64_t pre_total_size = total_size; for (int64_t i = 0; i < INTERVAL_NUM && OB_SUCC(ret); ++i) { if (i < LESS_THAN_100M_CNT) { // 1M total_size += LESS_THAN_100M_INTERVAL_SIZE; } else if (i < LESS_THAN_500M_CNT) { // 2M total_size += LESS_THAN_500M_INTERVAL_SIZE; } else if (i < LESS_THAN_1G_CNT) { // 5M total_size += LESS_THAN_1G_INTERVAL_SIZE; } else if (i < LESS_THAN_5G_CNT) { // 10M total_size += LESS_THAN_5G_INTERVAL_SIZE; } else if (i < LESS_THAN_10G_CNT) { // 50M total_size += LESS_THAN_10G_INTERVAL_SIZE; } else if (i < LESS_THAN_100G_CNT) { // 900M total_size += LESS_THAN_100G_INTERVAL_SIZE; } else if (i < LESS_THAN_1T_CNT) { // 9000M total_size += LESS_THAN_1T_INTERVAL_SIZE; } void* buf = static_cast(&sql_mem_mgr->wa_intervals_[i]); ObSqlWorkAreaInterval* wa_interval = new (buf) ObSqlWorkAreaInterval(i, total_size); ObWorkareaHistogram workarea_hist(pre_total_size, total_size); if (OB_FAIL(sql_mem_mgr->workarea_histograms_.push_back(workarea_hist))) { LOG_WARN("failed to push back workarea histogram", K(ret), K(i)); } UNUSED(wa_interval); pre_total_size = total_size; } if (MAX_INTERVAL_SIZE != total_size) { ret = OB_ERR_UNEXPECTED; LOG_ERROR("unexpect size", K(total_size)); } else { sql_mem_mgr->min_bound_size_ = MIN_GLOBAL_BOUND_SIZE; } if (OB_SUCC(ret)) { char* buf = reinterpret_cast(sql_mem_mgr->profile_lists_); for (int64_t i = 0; i < HASH_CNT; ++i) { ObSqlMemoryList* list = new (buf) ObSqlMemoryList(i); list->get_profile_list().reset(); buf += sizeof(ObSqlMemoryList); } } if (OB_SUCC(ret)) { if (OB_FAIL(sql_mem_mgr->wa_ht_.create(MAX_WORKAREA_STAT_CNT, common::ObModIds::OB_SQL_EXECUTOR, common::ObModIds::OB_SQL_EXECUTOR, tenant_id))) { LOG_WARN("failed to create hashmap", K(ret)); } else if (OB_FAIL(sql_mem_mgr->workarea_stats_.prepare_allocate(MAX_WORKAREA_STAT_CNT))) { LOG_WARN("failed to prepare element", K(ret)); } else { for (int64_t i = 0; i < MAX_WORKAREA_STAT_CNT; ++i) { ObSqlWorkAreaStat& wa_stat = sql_mem_mgr->workarea_stats_.at(i); wa_stat.set_seqno(i); } } } } if (OB_FAIL(ret)) { if (nullptr != sql_mem_mgr) { if (nullptr != sql_mem_mgr->wa_intervals_) { sql_mem_mgr->allocator_.free(sql_mem_mgr->wa_intervals_); sql_mem_mgr->wa_intervals_ = nullptr; } if (nullptr != sql_mem_mgr->profile_lists_) { sql_mem_mgr->allocator_.free(sql_mem_mgr->profile_lists_); sql_mem_mgr->profile_lists_ = nullptr; } sql_mem_mgr->wa_ht_.destroy(); sql_mem_mgr->workarea_stats_.reset(); sql_mem_mgr->workarea_histograms_.reset(); } } LOG_INFO("init sql memory manager", K(work_area_interval_size), K(tenant_id), K(ret)); } if (OB_FAIL(ret)) { if (nullptr != sql_mem_mgr) { sql_mem_mgr->allocator_.reset(); common::ob_delete(sql_mem_mgr); } sql_mem_mgr = nullptr; } } return ret; } void ObTenantSqlMemoryManager::mtl_destroy(ObTenantSqlMemoryManager*& sql_mem_mgr) { if (nullptr != sql_mem_mgr) { if (nullptr != sql_mem_mgr->wa_intervals_) { sql_mem_mgr->allocator_.free(sql_mem_mgr->wa_intervals_); sql_mem_mgr->wa_intervals_ = nullptr; } if (nullptr != sql_mem_mgr->profile_lists_) { sql_mem_mgr->allocator_.free(sql_mem_mgr->profile_lists_); sql_mem_mgr->profile_lists_ = nullptr; } sql_mem_mgr->wa_ht_.destroy(); sql_mem_mgr->workarea_stats_.reset(); sql_mem_mgr->workarea_histograms_.reset(); sql_mem_mgr->allocator_.reset(); common::ob_delete(sql_mem_mgr); } sql_mem_mgr = nullptr; } int ObTenantSqlMemoryManager::calc_work_area_size_by_profile(int64_t global_bound_size, ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; if (profile.is_hash_join_wa()) { if (global_bound_size >= profile.get_cache_size()) { profile.set_expect_size(profile.get_cache_size()); } else if (global_bound_size >= profile.get_one_pass_size()) { profile.set_expect_size(global_bound_size); } else if (global_bound_size < profile.get_min_size()) { profile.set_expect_size(profile.get_min_size()); } else { profile.set_expect_size(global_bound_size); } } else if (profile.is_sort_wa()) { if (global_bound_size > profile.get_cache_size()) { profile.set_expect_size(profile.get_cache_size()); } else if (global_bound_size > profile.get_one_pass_size()) { profile.set_expect_size(profile.get_one_pass_size()); } else if (global_bound_size < profile.get_min_size()) { profile.set_expect_size(profile.get_min_size()); } else { profile.set_expect_size(global_bound_size); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpect profile type", K(profile.get_work_area_type())); } profile.set_global_bound_size(global_bound_size); profile.set_max_bound(min(global_bound_size, profile.get_cache_size())); return ret; } int ObTenantSqlMemoryManager::get_work_area_size(ObIAllocator* allocator, ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; if (!profile.is_registered()) { // disable sql memory manager } else if (enable_auto_memory_mgr_) { increase(profile.get_cache_size()); LOG_TRACE("trace drift size", K(drift_size_), K(global_bound_size_)); if (need_manual_calc_bound()) { ++manual_calc_cnt_; if (OB_FAIL(calculate_global_bound_size(allocator, false))) { LOG_WARN("failed to calculate global bound size", K(global_bound_size_)); } else { LOG_TRACE("trace manual calc global bound size", K(global_bound_size_), K(profile.get_one_pass_size()), K(drift_size_), K(mem_target_)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(calc_work_area_size_by_profile(get_global_bound_size(), profile))) { LOG_WARN("failed to calculate worka area size by profile", K(ret), K(profile)); } } return ret; } int ObTenantSqlMemoryManager::register_work_area_profile(ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; if (!profile.is_registered()) { if (OB_NOT_NULL(profile.get_prev())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: next is null, but prev is not null", K(ret)); } else if (!ObSqlWorkAreaProfile::auto_sql_memory_manager(profile)) { // data is small, don't use auto memory manager } else { int64_t hash_val = get_hash_value(profile.get_id()); if (hash_val < 0 || hash_val >= HASH_CNT) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpect hash val", K(hash_val), K(profile)); } else if (OB_FAIL(profile_lists_[hash_val].register_work_area_profile(profile))) { LOG_WARN("failed to register work area profile", K(hash_val), K(profile)); } else { profile.active_time_ = ObTimeUtility::current_time(); } } } return ret; } int ObTenantSqlMemoryManager::update_work_area_profile( common::ObIAllocator* allocator, ObSqlWorkAreaProfile& profile, const int64_t delta_size) { int ret = OB_SUCCESS; UNUSED(profile); if (enable_auto_memory_mgr_ && profile.get_auto_policy()) { // delta_size maybe negative integer (ATOMIC_AAF(&drift_size_, delta_size)); if (need_manual_by_drift()) { int64_t pre_drift_size = drift_size_; ++manual_calc_cnt_; if (OB_ISNULL(allocator)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("allocator is null", K(lbt())); } else if (OB_FAIL(calculate_global_bound_size(allocator, false))) { LOG_WARN("failed to calculate global bound size", K(global_bound_size_)); } else { LOG_TRACE("trace manual calc global bound size by drift", K(global_bound_size_), K(profile.get_one_pass_size()), K(drift_size_), K(mem_target_), K(pre_drift_size)); } } } return ret; } int ObTenantSqlMemoryManager::fill_workarea_stat(ObSqlWorkAreaStat& wa_stat, ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; if (profile.get_operator_type() != wa_stat.get_op_type()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: operator type is not match", K(profile.get_operator_type()), K(wa_stat.get_op_type())); } else { wa_stat.est_cache_size_ = profile.get_cache_size(); wa_stat.est_one_pass_size_ = profile.get_one_pass_size(); wa_stat.last_memory_used_ = profile.get_max_mem_used(); wa_stat.last_execution_ = profile.get_number_pass(); wa_stat.last_degree_ = profile.get_dop(); int64_t num_executions = profile.get_number_pass(); if (0 == num_executions) { wa_stat.increase_optimal_executions(); } else if (1 == num_executions) { wa_stat.increase_onepass_executions(); } else { wa_stat.increase_multipass_executions(); } int64_t active_avg_time = wa_stat.get_total_executions() * wa_stat.get_active_avg_time(); wa_stat.increase_total_executions(); wa_stat.active_avg_time_ = (active_avg_time + (ObTimeUtility::current_time() - profile.get_active_time())) / wa_stat.get_total_executions(); wa_stat.last_temp_size_ = profile.get_dumped_size(); if (wa_stat.max_temp_size_ < wa_stat.last_temp_size_) { wa_stat.max_temp_size_ = wa_stat.last_temp_size_; } wa_stat.is_auto_policy_ = profile.get_auto_policy(); } return ret; } int ObTenantSqlMemoryManager::try_fill_workarea_stat( ObSqlWorkAreaStat::WorkareaKey& workarea_key, ObSqlWorkAreaProfile& profile, bool& need_insert) { int ret = OB_SUCCESS; need_insert = false; ObLatchRGuard guard(lock_, ObLatchIds::CONFIG_LOCK); ObSqlWorkAreaStat* wa_stat = nullptr; if (OB_FAIL(wa_ht_.get_refactored(workarea_key, wa_stat))) { if (OB_HASH_NOT_EXIST == ret) { need_insert = true; ret = OB_SUCCESS; } else { LOG_WARN("failed to get stat", K(ret)); } } else if (OB_ISNULL(wa_stat)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: wa stat is null", K(ret), K(profile)); } else { int64_t seqno = wa_stat->get_seqno(); if (seqno < 0 || seqno >= MAX_WORKAREA_STAT_CNT) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: seqno is invalid", K(ret), K(profile), K(seqno), K(*wa_stat)); } else { ObSqlWorkAreaStat& tmp_wa_stat = workarea_stats_.at(seqno); if (&tmp_wa_stat != wa_stat) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: wa stat is not match", K(ret), K(profile), K(seqno)); } else if (OB_FAIL(fill_workarea_stat(tmp_wa_stat, profile))) { LOG_WARN("failed to fill workarea stat", K(ret)); } } } return ret; } // write lock and create new stat int ObTenantSqlMemoryManager::new_and_fill_workarea_stat( ObSqlWorkAreaStat::WorkareaKey& workarea_key, ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; ObSqlWorkAreaStat* wa_stat = nullptr; ObLatchWGuard guard(lock_, ObLatchIds::CONFIG_LOCK); if (OB_FAIL(wa_ht_.get_refactored(workarea_key, wa_stat))) {} if (OB_HASH_NOT_EXIST == ret) { ret = OB_SUCCESS; if (is_wa_full()) { // eliminate one if (wa_start_ != wa_end_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: wa is full, but start and end position is not match", K(wa_start_), K(wa_end_), K(wa_cnt_)); } else { ObSqlWorkAreaStat* tmp_wa_stat = nullptr; wa_start_ = (wa_start_ + 1) % MAX_WORKAREA_STAT_CNT; wa_stat = &workarea_stats_.at(wa_end_); if (OB_FAIL(wa_ht_.erase_refactored(wa_stat->get_workarea_key(), &tmp_wa_stat))) { LOG_WARN("failed to erase workarea stat", K(ret), K(wa_stat->get_workarea_key())); } else if (wa_stat != tmp_wa_stat) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: wa stat is not match", K(ret)); } else { wa_stat->reset(); --wa_cnt_; } } } else { wa_stat = &workarea_stats_.at(wa_end_); wa_stat->reset(); } if (OB_FAIL(ret)) { } else { // new item: key + operator type wa_stat->workarea_key_.set_sql_id(profile.get_sql_id()); wa_stat->workarea_key_.set_plan_id(profile.get_plan_id()); wa_stat->workarea_key_.set_operator_id(profile.get_operator_id()); wa_stat->op_type_ = profile.get_operator_type(); if (OB_FAIL(fill_workarea_stat(*wa_stat, profile))) { LOG_WARN("failed to fill workarea stat", K(ret)); } else if (OB_FAIL(wa_ht_.set_refactored(workarea_key, wa_stat))) { LOG_WARN("failed to set refactored", K(ret)); } else { wa_end_ = (wa_end_ + 1) % MAX_WORKAREA_STAT_CNT; ++wa_cnt_; LOG_TRACE("new workarea stat:", K(wa_stat->workarea_key_), K(workarea_key), K(wa_stat->workarea_key_ == workarea_key), K(wa_stat->seqno_), K(profile), K(wa_cnt_), K(wa_start_), K(wa_end_)); } } } else if (OB_SUCC(ret)) { if (OB_ISNULL(wa_stat)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: wa_stat is null", K(ret)); } else if (OB_FAIL(fill_workarea_stat(*wa_stat, profile))) { LOG_WARN("failed to fill workarea stat", K(ret)); } } return ret; } int ObTenantSqlMemoryManager::collect_workarea_stat(ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; if (profile.has_exec_ctx()) { ObSqlWorkAreaStat::WorkareaKey workarea_key(profile.get_plan_id(), profile.get_operator_id()); workarea_key.set_sql_id(profile.get_sql_id()); bool need_insert = false; if (OB_FAIL(try_fill_workarea_stat(workarea_key, profile, need_insert))) { LOG_WARN("failed to try fill workarea stat", K(ret)); } else if (need_insert && OB_FAIL(new_and_fill_workarea_stat(workarea_key, profile))) { LOG_WARN("failed to create new and fill workarea start", K(ret)); } } return ret; } int ObTenantSqlMemoryManager::fill_workarea_histogram(ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; int64_t idx = INT64_MAX; int64_t size = INT64_MAX; int64_t max_mem_used = profile.get_mem_used(); if (OB_FAIL(find_interval_index(max_mem_used, idx, size))) { LOG_WARN("failed to find interval index", K(ret)); } else if (INT64_MAX == idx || INT64_MAX == size) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: can't found any interval", K(idx), K(size), K(profile)); } else { ObWorkareaHistogram& hist = workarea_histograms_.at(idx); if (max_mem_used < hist.get_low_optimal_size() || max_mem_used > hist.get_high_optimal_size()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: find interval error", K(ret), "mem used", max_mem_used, "low bound", hist.get_low_optimal_size(), "high bound", hist.get_high_optimal_size()); } else { if (0 == profile.get_number_pass()) { hist.increase_optimal_executions(); } else if (1 == profile.get_number_pass()) { hist.increase_onepass_executions(); } else if (1 < profile.get_number_pass()) { hist.increase_multipass_executions(); } hist.increase_total_executions(); } } return ret; } int ObTenantSqlMemoryManager::unregister_work_area_profile(ObSqlWorkAreaProfile& profile) { int ret = OB_SUCCESS; lib::ObMutexGuard guard(mutex_); if (profile.is_registered()) { int64_t hash_val = get_hash_value(profile.get_id()); if (hash_val < 0 || hash_val >= HASH_CNT) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpect hash val", K(hash_val), K(profile)); } else if (OB_FAIL(profile_lists_[hash_val].unregister_work_area_profile(profile))) { LOG_WARN("failed to register work area profile", K(hash_val), K(profile)); } else { if (enable_auto_memory_mgr_ && profile.get_auto_policy()) { decrease(profile.get_cache_size()); } if (!profile.need_profiled()) { } else if (OB_FAIL(collect_workarea_stat(profile))) { LOG_WARN("failed to fill workarea stat", K(ret)); } else if (OB_FAIL(fill_workarea_histogram(profile))) { LOG_WARN("failed to fill workarea histogram", K(ret)); } LOG_TRACE("unregister workarea profile", K(profile), K(ret)); } } return ret; } int ObTenantSqlMemoryManager::get_max_work_area_size(int64_t& max_wa_memory_size, const bool auto_calc) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObSysVarSchema* var_schema = NULL; ObObj value; int64_t pctg = 0; max_wa_memory_size = 0; if (OB_ISNULL(GCTX.schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null"); } else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(tenant_id_, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_system_variable( tenant_id_, SYS_VAR_OB_SQL_WORK_AREA_PERCENTAGE, var_schema))) { LOG_WARN("get tenant system variable failed", K(ret), K(tenant_id_)); } else if (OB_ISNULL(var_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("var_schema is null"); } else if (OB_FAIL(var_schema->get_value(NULL, NULL, value))) { LOG_WARN("get value from var_schema failed", K(ret), K(*var_schema)); } else if (OB_FAIL(value.get_int(pctg))) { LOG_WARN("get int from value failed", K(ret), K(value)); } else { int64_t tenant_max_memory_limit = get_tenant_memory_limit(tenant_id_); int64_t tenant_memory_hold = get_tenant_memory_hold(tenant_id_); int64_t tenant_work_area_max_size = tenant_max_memory_limit * pctg / 100; int64_t tenant_work_area_memory_hold = get_tenant_memory_hold(tenant_id_, common::ObCtxIds::WORK_AREA); int64_t max_tenant_memory_size = tenant_max_memory_limit - tenant_memory_hold; int64_t max_workarea_memory_size = tenant_work_area_max_size - tenant_work_area_memory_hold; if (max_workarea_memory_size > 0 && max_tenant_memory_size > 0 && max_workarea_memory_size > max_tenant_memory_size) { ObTenantResourceMgrHandle resource_handle; if (OB_FAIL(ObResourceMgr::get_instance().get_tenant_resource_mgr(tenant_id_, resource_handle))) { ret = OB_SUCCESS; } else { // TODO: kvcache大概可以淘汰多少内存,目前没有数据,后续寒晖他们会提供接口 // bug34818894 https://work.aone.alibaba-inc.com/issue/34818894 // 这里暂时写一个默认比例 max_tenant_memory_size += resource_handle.get_memory_mgr()->get_cache_hold() * pctg; } } int64_t remain_memory_size = max_tenant_memory_size > 0 ? min(max_workarea_memory_size, max_tenant_memory_size) : max_tenant_memory_size; int64_t total_alloc_size = sql_mem_callback_.get_total_alloc_size(); double ratio = total_alloc_size * 1.0 / tenant_work_area_memory_hold; int64_t pre_mem_target = mem_target_; double hold_ratio = 1. * tenant_work_area_memory_hold / tenant_work_area_max_size; int64_t tmp_max_wa_memory_size = (remain_memory_size > 0) ? (1 - hold_ratio * hold_ratio) * remain_memory_size + total_alloc_size : total_alloc_size; double alloc_ratio = total_alloc_size * 1.0 / tmp_max_wa_memory_size; if (total_alloc_size >= tmp_max_wa_memory_size) { max_wa_memory_size = (tmp_max_wa_memory_size >> 1); } else { max_wa_memory_size = tmp_max_wa_memory_size * (1 - alloc_ratio * alloc_ratio); } max_workarea_size_ = tenant_work_area_max_size; workarea_hold_size_ = tenant_work_area_memory_hold; max_auto_workarea_size_ = max_wa_memory_size; if (0 > max_wa_memory_size) { max_wa_memory_size = 0; LOG_INFO("max work area is 0", K(tenant_max_memory_limit), K(total_alloc_size), K(tenant_work_area_memory_hold), K(tenant_work_area_max_size)); } if (auto_calc) { LOG_INFO("trace max work area", K(tenant_max_memory_limit), K(total_alloc_size), K(tenant_work_area_memory_hold), K(tenant_work_area_max_size), K(max_wa_memory_size), K(tmp_max_wa_memory_size), K(pre_mem_target), K(remain_memory_size), K(ratio), K(alloc_ratio), K(hold_ratio), K(tenant_memory_hold)); } } return ret; } int ObTenantSqlMemoryManager::find_interval_index(const int64_t cache_size, int64_t& idx, int64_t& out_cache_size) { int ret = OB_SUCCESS; bool found = false; idx = -1; int64_t total_size = cache_size; out_cache_size = cache_size; if (cache_size <= wa_intervals_[LESS_THAN_100M_CNT - 1].get_interval_cache_size()) { idx = (total_size - 1) / LESS_THAN_100M_INTERVAL_SIZE; found = true; } else { idx = LESS_THAN_100M_CNT; total_size -= LESS_THAN_100M_INTERVAL_SIZE * LESS_THAN_100M_CNT; if (cache_size <= wa_intervals_[LESS_THAN_500M_CNT - 1].get_interval_cache_size()) { idx += (total_size - 1) / LESS_THAN_500M_INTERVAL_SIZE; found = true; } else { idx = LESS_THAN_500M_CNT; total_size -= LESS_THAN_500M_INTERVAL_SIZE * (LESS_THAN_500M_CNT - LESS_THAN_100M_CNT); } } if (found) { } else if (cache_size <= wa_intervals_[LESS_THAN_1G_CNT - 1].get_interval_cache_size()) { idx += (total_size - 1) / LESS_THAN_1G_INTERVAL_SIZE; found = true; } else { idx = LESS_THAN_1G_CNT; total_size -= LESS_THAN_1G_INTERVAL_SIZE * (LESS_THAN_1G_CNT - LESS_THAN_500M_CNT); if (cache_size <= wa_intervals_[LESS_THAN_5G_CNT - 1].get_interval_cache_size()) { idx += (total_size - 1) / LESS_THAN_5G_INTERVAL_SIZE; found = true; } else { idx = LESS_THAN_5G_CNT; total_size -= LESS_THAN_5G_INTERVAL_SIZE * (LESS_THAN_5G_CNT - LESS_THAN_1G_CNT); } } if (found) { } else if (cache_size <= wa_intervals_[LESS_THAN_10G_CNT - 1].get_interval_cache_size()) { idx += (total_size - 1) / LESS_THAN_10G_INTERVAL_SIZE; found = true; } else { idx = LESS_THAN_10G_CNT; total_size -= LESS_THAN_10G_INTERVAL_SIZE * (LESS_THAN_10G_CNT - LESS_THAN_5G_CNT); if (cache_size <= wa_intervals_[LESS_THAN_100G_CNT - 1].get_interval_cache_size()) { idx += (total_size - 1) / LESS_THAN_100G_INTERVAL_SIZE; found = true; } else { idx = LESS_THAN_100G_CNT; total_size -= LESS_THAN_100G_INTERVAL_SIZE * (LESS_THAN_100G_CNT - LESS_THAN_10G_CNT); if (cache_size <= wa_intervals_[LESS_THAN_1T_CNT - 1].get_interval_cache_size()) { idx += (total_size - 1) / LESS_THAN_1T_INTERVAL_SIZE; found = true; } else { found = true; idx = INTERVAL_NUM - 1; LOG_TRACE("too big size", K(cache_size)); } } } // check if (found) { if (cache_size < 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: cache size is less than 0", K(idx), K(cache_size), K(ret)); } else if (INTERVAL_NUM - 1 != idx) { if (0 == idx) { if (cache_size > wa_intervals_[idx].get_interval_cache_size()) { ret = OB_ERR_UNEXPECTED; LOG_WARN( "failed to find interval index", K(idx), K(cache_size), K(wa_intervals_[idx].get_interval_cache_size())); } } else { if (cache_size <= wa_intervals_[idx - 1].get_interval_cache_size() || cache_size > wa_intervals_[idx].get_interval_cache_size()) { ret = OB_ERR_UNEXPECTED; LOG_WARN( "failed to find interval index", K(idx), K(cache_size), K(wa_intervals_[idx].get_interval_cache_size())); } } } else { if (cache_size <= wa_intervals_[idx - 1].get_interval_cache_size()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to find interval index", K(idx), K(cache_size), K(wa_intervals_[idx - 1].get_interval_cache_size())); } } } return ret; } // sum memory size of all profiles int ObTenantSqlMemoryManager::count_profile_into_work_area_intervals( ObSqlWorkAreaInterval* wa_intervals, int64_t& total_memory_size, int64_t& cur_profile_cnt) { int ret = OB_SUCCESS; int64_t interval_idx = -1; int64_t one_pass_idx = -1; int64_t cache_size = 0; int64_t one_pass_size = 0; total_memory_size = 0; cur_profile_cnt = 0; // count interval stat from all profiles if (nullptr == profile_lists_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("profile list is null", K(ret)); } else { for (int64_t i = 0; i < HASH_CNT && OB_SUCC(ret); ++i) { ObLockGuard lock_guard(profile_lists_[i].get_lock()); ObDList& profile_list = profile_lists_[i].get_profile_list(); DLIST_FOREACH_X(profile, profile_list, OB_SUCC(ret)) { if (!profile->get_auto_policy()) { } else if (OB_FAIL(find_interval_index(profile->get_cache_size(), interval_idx, cache_size))) { LOG_WARN("failed to find interval index", K(*profile)); } else { one_pass_size = profile->calc_one_pass_size(cache_size); if (OB_FAIL(find_interval_index(profile->get_one_pass_size(), one_pass_idx, one_pass_size))) { LOG_WARN("failed to find interval index", K(*profile)); } else if (OB_FAIL(wa_intervals[interval_idx].get_interval_stat().analyze_profile( *profile, cache_size, one_pass_size, MAX_INTERVAL_SIZE))) { LOG_WARN("failed to analyze profile", K(*profile)); } else if (OB_FAIL(wa_intervals[one_pass_idx].get_interval_stat().analyze_profile( *profile, one_pass_size, 0, MAX_INTERVAL_SIZE, true))) { LOG_WARN("failed to analyze profile", K(*profile)); } else { total_memory_size += cache_size; ++cur_profile_cnt; } } } } } return ret; } bool ObTenantSqlMemoryManager::enable_auto_sql_memory_manager() { bool auto_memory_mgr = false; ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id_)); if (tenant_config.is_valid()) { const ObString tmp_str(tenant_config->workarea_size_policy.str()); auto_memory_mgr = !tmp_str.case_compare("AUTO"); LOG_TRACE("get work area policy config", K(tenant_id_), K(auto_memory_mgr), K(tmp_str), K(tenant_config->workarea_size_policy.str())); } else { LOG_WARN("failed to init tenant config", K(tenant_id_)); } return auto_memory_mgr; } // ensure lock outside void ObTenantSqlMemoryManager::reset() { if (nullptr != profile_lists_) { for (int64_t i = 0; i < HASH_CNT; ++i) { profile_lists_[i].reset(); } } // sql_mem_callback_.reset(); drift_size_ = 0; profile_cnt_ = 0; global_bound_size_ = 0; } // try best to push global_bound_size to every profile registered int ObTenantSqlMemoryManager::try_push_profiles_work_area_size(int64_t global_bound_size) { int ret = OB_SUCCESS; if (nullptr == profile_lists_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("profile list is null", K(ret)); } else { for (int64_t i = 0; i < HASH_CNT && OB_SUCC(ret); ++i) { if (OB_SUCC(profile_lists_[i].get_lock().trylock())) { ObDList& profile_list = profile_lists_[i].get_profile_list(); DLIST_FOREACH_X(profile, profile_list, OB_SUCC(ret)) { if (profile->get_auto_policy() && OB_FAIL(calc_work_area_size_by_profile(global_bound_size, *profile))) { ret = OB_SUCCESS; LOG_WARN("failed to calculate worka area size by profile", K(ret), K(*profile), K(global_bound_size)); } } profile_lists_[i].get_lock().unlock(); } else { ret = OB_SUCCESS; } } } return OB_SUCCESS; } int ObTenantSqlMemoryManager::calculate_global_bound_size_by_interval_info( ObIAllocator& allocator, const int64_t wa_max_memory_size, const bool auto_calc) { int ret = OB_SUCCESS; ObSqlWorkAreaCalcInfo calc_info; if (OB_FAIL(calc_info.init(allocator, wa_intervals_, INTERVAL_NUM))) { LOG_WARN("failed to init work area calc info", K(ret)); } else { int64_t pre_profile_cnt = profile_cnt_; int64_t total_memory_size = 0; int64_t cur_profile_cnt = 0; if (OB_FAIL( count_profile_into_work_area_intervals(calc_info.get_wa_intervals(), total_memory_size, cur_profile_cnt))) { LOG_WARN("failed to count profiles", K(ret)); } else if (OB_FAIL(calc_info.calculate_global_bound_size( wa_max_memory_size, total_memory_size, pre_profile_cnt, auto_calc))) { LOG_WARN("failed to find best interval index", K(ret), K(wa_max_memory_size), K(total_memory_size)); } else { int64_t pre_drift_size = drift_size_; { lib::ObMutexGuard guard(mutex_); drift_size_ = 0; pre_profile_cnt_ = cur_profile_cnt; global_bound_size_ = calc_info.get_global_bound_size(); mem_target_ = calc_info.get_mem_target(); pre_enable_auto_memory_mgr_ = true; // last set enable auto memory manager, so others read the vairable to avoiding dirty read enable_auto_memory_mgr_ = true; } if (auto_calc) { LOG_INFO("timer to calc global bound size", K(ret), K(global_bound_size_), K(manual_calc_cnt_), K(drift_size_), K(pre_drift_size), K(wa_max_memory_size), K(sql_mem_callback_.get_total_alloc_size()), K(tenant_id_), K(profile_cnt_), K(pre_profile_cnt_), K(pre_profile_cnt), K(calc_info.get_global_bound_size()), K(total_memory_size), K(cur_profile_cnt), K(calc_info.get_mem_target()), K(auto_calc), K(sql_mem_callback_.get_total_dump_size())); } if (OB_FAIL(try_push_profiles_work_area_size(calc_info.get_global_bound_size()))) { LOG_WARN("failed to push profiles work area size", K(ret), K(calc_info.get_global_bound_size())); } } } calc_info.destroy(allocator); return ret; } int ObTenantSqlMemoryManager::calculate_global_bound_size(ObIAllocator* allocator, bool auto_calc) { int ret = OB_SUCCESS; int64_t wa_max_memory_size = 0; // set enable_auto_sql_memory_mgr after calculate global bound size bool auto_memory_mgr = enable_auto_sql_memory_manager(); if (!auto_memory_mgr) { // manually memory manager lib::ObMutexGuard guard(mutex_); enable_auto_memory_mgr_ = false; if (pre_enable_auto_memory_mgr_) { reset(); pre_enable_auto_memory_mgr_ = false; } } else { { lib::ObMutexGuard guard(mutex_); if (!pre_enable_auto_memory_mgr_) { if (enable_auto_memory_mgr_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status", K(ret)); } else { reset(); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(get_max_work_area_size(wa_max_memory_size, auto_calc))) { LOG_WARN("failed to get percent"); } else if (0 == wa_max_memory_size) { lib::ObMutexGuard guard(mutex_); global_bound_size_ = min_bound_size_; drift_size_ = 0; pre_enable_auto_memory_mgr_ = true; // last set enable auto memory manager, so others read the vairable to avoiding dirty read enable_auto_memory_mgr_ = true; if (auto_calc) { LOG_INFO("work area memory zero", K(tenant_id_), K(global_bound_size_)); } } else { if (OB_ISNULL(allocator)) { allocator = &allocator_; } if (OB_FAIL(calculate_global_bound_size_by_interval_info(*allocator, wa_max_memory_size, auto_calc))) { LOG_WARN("failed to calculate global bound size", K(ret)); } } } return ret; } int ObTenantSqlMemoryManager::get_workarea_stat(ObIArray& wa_stats) { int ret = OB_SUCCESS; ObLatchRGuard guard(lock_, ObLatchIds::CONFIG_LOCK); for (int64_t i = wa_start_; i < wa_start_ + wa_cnt_ && OB_SUCC(ret); ++i) { int64_t nth = i % MAX_WORKAREA_STAT_CNT; if (OB_FAIL(wa_stats.push_back(workarea_stats_.at(nth)))) { LOG_WARN("failed to push back workarea stat", K(ret)); } else { LOG_TRACE("trace workarea history", K(workarea_stats_.at(i)), K(wa_stats.at(wa_stats.count() - 1))); } } return ret; } int ObTenantSqlMemoryManager::get_workarea_histogram(common::ObIArray& wa_histograms) { int ret = OB_SUCCESS; int64_t cnt = workarea_histograms_.count(); for (int64_t i = 0; i < cnt && OB_SUCC(ret); ++i) { if (OB_FAIL(wa_histograms.push_back(workarea_histograms_.at(i)))) { LOG_WARN("failed to push back workarea stat", K(ret)); } else { LOG_TRACE( "trace workarea histogram", K(workarea_histograms_.at(i)), K(wa_histograms.at(wa_histograms.count() - 1))); } } return ret; } int ObTenantSqlMemoryManager::get_all_active_workarea(ObIArray& wa_actives) { int ret = OB_SUCCESS; if (nullptr == profile_lists_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("profile list is null", K(ret)); } else { for (int64_t i = 0; i < HASH_CNT && OB_SUCC(ret); ++i) { ObLockGuard lock_guard(profile_lists_[i].get_lock()); ObDList& profile_list = profile_lists_[i].get_profile_list(); DLIST_FOREACH_X(profile, profile_list, OB_SUCC(ret)) { ObSqlWorkareaProfileInfo profile_info; profile_info.profile_ = *profile; profile_info.plan_id_ = profile->get_plan_id(); profile_info.sql_exec_id_ = profile->get_exec_id(); profile_info.set_sql_id(profile->get_sql_id()); profile_info.session_id_ = profile->get_session_id(); if (OB_FAIL(wa_actives.push_back(profile_info))) { LOG_WARN("failed to push back profile", K(ret)); } } } } return ret; } int ObTenantSqlMemoryManager::get_workarea_memory_info(ObSqlWorkareaCurrentMemoryInfo& memory_info) { int ret = OB_SUCCESS; memory_info.enable_ = enable_auto_memory_mgr_; memory_info.max_workarea_size_ = max_workarea_size_; memory_info.workarea_hold_size_ = workarea_hold_size_; memory_info.max_auto_workarea_size_ = max_auto_workarea_size_; memory_info.mem_target_ = mem_target_; memory_info.total_mem_used_ = sql_mem_callback_.get_total_alloc_size(); memory_info.global_bound_size_ = global_bound_size_; memory_info.drift_size_ = drift_size_; memory_info.workarea_cnt_ = profile_cnt_; memory_info.manual_calc_cnt_ = manual_calc_cnt_; return ret; } } // namespace sql } // namespace oceanbase