[CP]optimize gather database stats job proc

This commit is contained in:
Larry955 2023-08-14 10:48:43 +00:00 committed by ob-robot
parent c90faddb7b
commit e8f28b7c7b
6 changed files with 292 additions and 196 deletions

View File

@ -5161,132 +5161,179 @@ int ObDbmsStats::gather_database_stats_job_proc(sql::ObExecContext &ctx,
UNUSED(result);
const int64_t start_time = ObTimeUtility::current_time();
ObOptStatTaskInfo task_info;
ObGatherTableStatsHelper helper;
number::ObNumber num_duration;
int64_t duration_time = -1;
int64_t succeed_cnt = 0;
if (OB_FAIL(check_statistic_table_writeable(ctx))) {
ret = OB_SUCCESS;
LOG_INFO("auto gather database statistics abort because of statistic table is unwriteable");
} else if (lib::is_oracle_mode() && !params.empty() && !params.at(0).is_null() &&
OB_FAIL(params.at(0).get_number(num_duration))) {
LOG_WARN("failed to get duration", K(ret), K(params.at(0)));
} else if (lib::is_oracle_mode() && !params.empty() && !params.at(0).is_null() &&
OB_FAIL(num_duration.extract_valid_int64_with_trunc(duration_time))) {
LOG_WARN("extract_valid_int64_with_trunc failed", K(ret), K(num_duration));
} else if (lib::is_mysql_mode() && !params.empty() && !params.at(0).is_null() &&
OB_FAIL(params.at(0).get_int(duration_time))) {
LOG_WARN("failed to get duration", K(ret), K(params.at(0)));
} else if (OB_FAIL(ObOptStatMonitorManager::flush_database_monitoring_info(ctx))) {
LOG_WARN("failed to flush database monitoring info", K(ret));
} else if (OB_FAIL(helper.get_duration_time(params))) {
LOG_WARN("failed to get duration time");
} else if (OB_FAIL(get_need_statistics_tables(ctx, helper))) {
LOG_WARN("failed to get need statistics tables", K(ret));
} else if (helper.need_gather_table_stats()) {
int64_t total_cnt = helper.stat_tables_.count();
if (OB_FAIL(init_gather_task_info(ctx, ObOptStatGatherType::AUTO_GATHER, start_time, total_cnt, task_info))) {
LOG_WARN("failed to init gather task info", K(ret));
} else if (OB_FAIL(gather_tables_stats_with_default_param(ctx, helper, task_info))) {
LOG_WARN("failed to gather tables tats with default param");
} else {/*do nothing*/}
const int64_t exe_time = ObTimeUtility::current_time() - start_time;
LOG_INFO("have been gathered database stats job",
"the total used time:", exe_time,
"the duration time:", helper.duration_time_,
"the toatal gather table cnt:", total_cnt,
"the succeed to gather table cnt:", helper.succeed_count_,
"the failed to gather table cnt:", helper.failed_count_, K(ret));
//reset the error code, the reason is that the total gather time is reach the duration time.
ret = ret == OB_TIMEOUT ? OB_SUCCESS : ret;
task_info.task_end_time_ = ObTimeUtility::current_time();
task_info.ret_code_ = ret;
task_info.failed_count_ = helper.failed_count_;
ObOptStatManager::get_instance().update_opt_stat_task_stat(task_info);
}
} else if (OB_FAIL(init_gather_task_info(ctx, ObOptStatGatherType::AUTO_GATHER, start_time, 0, task_info))) {
LOG_WARN("failed to init gather task info", K(ret));
} else if (OB_FAIL(gather_database_table_stats(ctx, duration_time, succeed_cnt, task_info))) {
LOG_WARN("failed to gather table stats", K(ret));
} else {/*do nothing*/}
const int64_t exe_time = ObTimeUtility::current_time() - start_time;
LOG_INFO("have been gathered database stats job",
"the total used time:", exe_time,
"the duration time:", duration_time,
"the toatal gather table cnt:", task_info.task_table_count_,
"the succeed to gather table cnt:", succeed_cnt,
"the failed to gather table cnt:", task_info.failed_count_, K(ret));
//reset the error code, the reason is that the total gather time is reach the duration time.
ret = ret == OB_TIMEOUT ? OB_SUCCESS : ret;
task_info.task_end_time_ = ObTimeUtility::current_time();
task_info.ret_code_ = ret;
ObOptStatManager::get_instance().update_opt_stat_task_stat(task_info);
return ret;
}
int ObDbmsStats::get_need_statistics_tables(sql::ObExecContext &ctx, ObGatherTableStatsHelper &helper)
int ObDbmsStats::gather_database_table_stats(sql::ObExecContext &ctx,
const int64_t duration_time,
int64_t &succeed_cnt,
ObOptStatTaskInfo &task_info)
{
int ret = OB_SUCCESS;
ObSEArray<int64_t, 128> table_ids;
ObSchemaGetterGuard *schema_guard = ctx.get_virtual_table_ctx().schema_guard_;
ObSQLSessionInfo *session = ctx.get_my_session();
ObSEArray<const ObDatabaseSchema *, 16> database_schemas;
uint64_t tenant_id = OB_INVALID_ID;
uint64_t database_id = OB_INVALID_ID;
const ObDatabaseSchema *database_schema = NULL;
const ObTableSchema *table_schema = NULL;
if (OB_ISNULL(schema_guard) || OB_ISNULL(session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(schema_guard), K(session));
} else if (OB_FALSE_IT(tenant_id = session->get_effective_tenant_id())) {
} else if (is_virtual_tenant_id(tenant_id)) {
// do nothing
} else if (OB_FAIL(schema_guard->get_database_schemas_in_tenant(tenant_id, database_schemas))) {
LOG_WARN("failed to get database sehcmas in tenant", K(ret));
} else if (OB_FAIL(ObBasicStatsEstimator::get_need_stats_table_cnt(ctx, tenant_id,
task_info.task_table_count_))) {
LOG_WARN("failed to get all tables count", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < database_schemas.count(); ++i) {
ObSEArray<const ObTableSchema *, 128> table_schemas;
if (OB_ISNULL(database_schema = database_schemas.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(database_schema));
} else if (is_recyclebin_database_id(database_id = database_schema->get_database_id())) {
// do not gather statistics for tables in recyclebin
} else if (OB_FAIL(schema_guard->get_table_schemas_in_database(tenant_id,
database_id,
table_schemas))) {
LOG_WARN("failed to get table schema in database", K(ret));
} else {
for (int64_t j = 0; OB_SUCC(ret) && j < table_schemas.count(); ++j) {
bool is_valid = false;
if (OB_ISNULL(table_schema = table_schemas.at(j))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(table_schema));
} else if (OB_FAIL(ObDbmsStatsUtils::check_is_stat_table(*schema_guard,
tenant_id,
table_schema->get_table_id(),
is_valid))) {
LOG_WARN("failed to check sy table validity", K(ret));
} else if (!is_valid || table_schema->is_external_table()) {
// only gather statistics for following tables:
// 1. user table
// 2. valid sys table
// 3. virtual table
int64_t slice_cnt = 10000; // maximum tables we can gather stats at each iteration
int64_t tmp_succeed = 0;
do {
table_ids.reuse();
tmp_succeed = succeed_cnt;
if (OB_FAIL(ObBasicStatsEstimator::get_need_stats_tables(ctx, tenant_id, table_ids, slice_cnt))) {
LOG_WARN("failed to get tables that need gather stats", K(ret));
} else if (OB_FAIL(do_gather_tables_stats(ctx, *schema_guard, tenant_id, table_ids,
duration_time, succeed_cnt, task_info))) {
LOG_WARN("failed to gather table stats", K(ret));
}
LOG_INFO("succeed to gather table stats", K(ret), K(table_ids.count()), K(slice_cnt),
K(tmp_succeed), K(duration_time), K(succeed_cnt));
// case that we can break the loop:
// 1. #table_ids < slice_cnt, which means that we have fetched all the tables we need to gather stats
// 2. duration_time_ = -1, and has reached the ob_query_timeout session variable limit
// 3. duration_time is not -1, and the time we cost to gather stats has reached duration_time
} while (OB_SUCC(ret) && table_ids.count() == slice_cnt && (succeed_cnt - tmp_succeed) != 0);
// gather virtual table stats
ObSEArray<uint64_t, 256> all_table_ids;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(schema_guard->get_table_ids_in_tenant(tenant_id, all_table_ids))){
LOG_WARN("failed to get virtual table ids in tenant", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < all_table_ids.count(); ++i) {
int64_t table_id = static_cast<int64_t>(all_table_ids.at(i));
if (is_virtual_table(table_id) && !ObDbmsStatsUtils::is_no_stat_virtual_table(table_id)) {
if (OB_FAIL(do_gather_table_stats(ctx, *schema_guard, table_id, tenant_id,
duration_time, succeed_cnt, task_info))) {
LOG_WARN("failed to gather virtual table stats", K(ret));
} else {
ObStatTableWrapper wrapper;
StatTable stat_table(database_id, table_schema->get_table_id());
double stale_percent_threshold = OPT_DEFAULT_STALE_PERCENT;
bool is_big_table = false;
if (OB_FAIL(get_table_stale_percent_threshold(ctx,
tenant_id,
stat_table.table_id_,
stale_percent_threshold))) {
LOG_WARN("failed to get table stale percent threshold", K(ret));
} else if (OB_FAIL(get_table_stale_percent(ctx, tenant_id,
*table_schema,
stale_percent_threshold,
stat_table,
is_big_table))) {
LOG_WARN("failed to get table stale percent", K(ret));
} else if (OB_FAIL(ObBasicStatsEstimator::get_gather_table_duration(ctx,
tenant_id,
table_schema->get_table_id(),
wrapper.last_gather_duration_))) {
LOG_WARN("failed to get gather table duration");
} else if (stat_table.stale_percent_ < 0) {
wrapper.stat_type_ = ObStatType::ObFirstTimeToGather;
} else if (stat_table.stale_percent_ > stale_percent_threshold) {
wrapper.stat_type_ = ObStatType::ObStale;
} else {
wrapper.stat_type_ = ObStatType::ObNotStale;
}
if (OB_SUCC(ret)) {
wrapper.table_type_ = table_schema->is_user_table() ?
ObStatTableType::ObUserTable : ObStatTableType::ObSysTable;
wrapper.is_big_table_ = is_big_table;
if (wrapper.stat_type_ == ObStatType::ObNotStale) {
// do nothing
} else if (OB_FAIL(wrapper.stat_table_.assign(stat_table))) {
LOG_WARN("failed to assign stat table");
} else if (OB_FAIL(helper.stat_tables_.push_back(wrapper))) {
LOG_WARN("failed to push back stat tables");
}
}
++task_info.task_table_count_;
}
}
}
}
if (OB_SUCC(ret) && !helper.stat_tables_.empty()) {
std::sort(&helper.stat_tables_.at(0), &helper.stat_tables_.at(0) + helper.stat_tables_.count());
}
return ret;
}
int ObDbmsStats::do_gather_tables_stats(sql::ObExecContext &ctx,
ObSchemaGetterGuard &schema_guard,
const uint64_t tenant_id,
const ObIArray<int64_t> &table_ids,
const int64_t duration_time,
int64_t &succeed_cnt,
ObOptStatTaskInfo &task_info)
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < table_ids.count(); ++i) {
if (OB_FAIL(do_gather_table_stats(ctx, schema_guard, table_ids.at(i), tenant_id,
duration_time, succeed_cnt, task_info))) {
LOG_WARN("failed to gather table stats", K(ret));
}
}
return ret;
}
int ObDbmsStats::do_gather_table_stats(sql::ObExecContext &ctx,
ObSchemaGetterGuard &schema_guard,
const int64_t table_id,
const uint64_t tenant_id,
const int64_t duration_time,
int64_t &succeed_cnt,
ObOptStatTaskInfo &task_info)
{
int ret = OB_SUCCESS;
bool is_valid = false;
const ObTableSchema *table_schema = NULL;
if (OB_FAIL(ObDbmsStatsUtils::check_is_stat_table(schema_guard, tenant_id, table_id, is_valid))) {
LOG_WARN("failed to check sy table validity", K(ret));
} else if (!is_valid) {
// only gather statistics for following tables:
// 1. user table
// 2. valid sys table
// 3. virtual table
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, table_schema))) {
LOG_WARN("failed to get table schema", K(ret));
} else if (OB_ISNULL(table_schema)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else {
StatTable stat_table(table_schema->get_database_id(), table_id);
double stale_percent_threshold = OPT_DEFAULT_STALE_PERCENT;
if (OB_FAIL(get_table_stale_percent_threshold(ctx,
tenant_id,
table_schema->get_table_id(),
stale_percent_threshold))) {
LOG_WARN("failed to get table stale percent threshold", K(ret));
} else if (OB_FAIL(get_table_stale_percent(ctx, tenant_id, *table_schema,
stale_percent_threshold, stat_table))) {
LOG_WARN("failed to get table stale percent", K(ret));
} else if (stat_table.stale_percent_ < 0 || stat_table.stale_percent_ > stale_percent_threshold) {
if (is_oceanbase_sys_database_id(stat_table.database_id_)) {
lib::CompatModeGuard compat_guard(lib::Worker::CompatMode::MYSQL);
if (OB_FAIL(gather_table_stats_with_default_param(ctx, duration_time, stat_table, task_info))) {
LOG_WARN("failed to gather table stats with default param", K(ret));
}
} else if (OB_FAIL(gather_table_stats_with_default_param(ctx, duration_time, stat_table, task_info))) {
LOG_WARN("failed to gather table stats with default param", K(ret));
}
if (OB_FAIL(ret)) {
if (OB_ERR_QUERY_INTERRUPTED == ret) {
LOG_WARN("query interrupted", K(ret));
} else if (OB_TABLE_NOT_EXIST == ret || OB_TIMEOUT == ret) {
++task_info.failed_count_;
// do nothing
ret = OB_SUCCESS;
} else {
++task_info.failed_count_;
LOG_WARN("failed to gather table stats with some unknown reason", K(ret));
ret = OB_SUCCESS;
}
} else {
++succeed_cnt;
}
}
}
return ret;
@ -5296,15 +5343,14 @@ int ObDbmsStats::get_table_stale_percent(sql::ObExecContext &ctx,
const uint64_t tenant_id,
const ObTableSchema &table_schema,
const double stale_percent_threshold,
StatTable &stat_table,
bool &is_big_table)
StatTable &stat_table)
{
int ret = OB_SUCCESS;
uint64_t table_id = table_schema.get_table_id();
const int64_t part_id = PARTITION_LEVEL_ZERO == table_schema.get_part_level() ? table_id : -1;
ObSEArray<ObPartitionStatInfo, 4> partition_stat_infos;
bool is_locked = false;
is_big_table = false;
bool dummy_is_big_table = false;
if (OB_FAIL(ObBasicStatsEstimator::check_table_statistics_state(ctx,
tenant_id,
table_id,
@ -5321,11 +5367,12 @@ int ObDbmsStats::get_table_stale_percent(sql::ObExecContext &ctx,
stale_percent_threshold,
partition_stat_infos,
stat_table,
is_big_table))) {
dummy_is_big_table))) {
LOG_WARN("faild to get user partition table stale percent", K(ret));
} else {/*do nothing*/}
} else if (OB_FAIL(get_common_table_stale_percent(ctx, tenant_id, table_schema,
partition_stat_infos, stat_table, is_big_table))) {
partition_stat_infos, stat_table,
dummy_is_big_table))) {
LOG_WARN("failed to get common table stale percent", K(ret));
} else {/*do nothing*/}
return ret;
@ -5537,45 +5584,6 @@ int ObDbmsStats::get_user_partition_table_stale_percent(
return ret;
}
int ObDbmsStats::gather_tables_stats_with_default_param(ObExecContext &ctx,
ObGatherTableStatsHelper &helper,
ObOptStatTaskInfo &task_info)
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < helper.stat_tables_.count(); ++i) {
if (is_oceanbase_sys_database_id(helper.stat_tables_.at(i).stat_table_.database_id_)) {
lib::CompatModeGuard compat_guard(lib::Worker::CompatMode::MYSQL);
if (OB_FAIL(gather_table_stats_with_default_param(ctx,
helper.duration_time_,
helper.stat_tables_.at(i).stat_table_,
task_info))) {
LOG_WARN("failed to gather table stats with default param", K(ret));
}
} else if (OB_FAIL(gather_table_stats_with_default_param(ctx,
helper.duration_time_,
helper.stat_tables_.at(i).stat_table_,
task_info))) {
LOG_WARN("failed to gather table stats with default param", K(ret));
}
if (OB_FAIL(ret)) {
if (OB_ERR_QUERY_INTERRUPTED == ret) {
LOG_WARN("query interrupted", K(ret));
} else if (OB_TABLE_NOT_EXIST == ret || OB_TIMEOUT == ret) {
// do nothing
++helper.failed_count_;
ret = OB_SUCCESS;
} else {
++helper.failed_count_;
LOG_WARN("failed to gather table stats with some unknown reason", K(ret));
ret = OB_SUCCESS;
}
} else {
++helper.succeed_count_;
}
}
return ret;
}
int ObDbmsStats::gather_table_stats_with_default_param(ObExecContext &ctx,
const int64_t duration_time,
const StatTable &stat_table,

View File

@ -437,19 +437,32 @@ public:
sql::ParamStore &params,
common::ObObj &result);
static int get_need_statistics_tables(sql::ObExecContext &ctx,
ObGatherTableStatsHelper &helper);
static int gather_database_table_stats(sql::ObExecContext &ctx,
const int64_t duration_time,
int64_t &succeed_cnt,
ObOptStatTaskInfo &task_info);
static int do_gather_table_stats(sql::ObExecContext &ctx,
ObSchemaGetterGuard &schema_guard,
const int64_t table_id,
const uint64_t tenant_id,
const int64_t duration_time,
int64_t &succeed_cnt,
ObOptStatTaskInfo &task_info);
static int do_gather_tables_stats(sql::ObExecContext &ctx,
ObSchemaGetterGuard &schema_guard,
const uint64_t tenant_id,
const ObIArray<int64_t> &table_ids,
const int64_t duration_time,
int64_t &succeed_cnt,
ObOptStatTaskInfo &task_info);
static int get_table_stale_percent(sql::ObExecContext &ctx,
const uint64_t tenant_id,
const share::schema::ObTableSchema &table_schema,
const double stale_percent_threshold,
StatTable &stat_table,
bool &is_big_table);
static int gather_tables_stats_with_default_param(ObExecContext &ctx,
ObGatherTableStatsHelper &helper,
ObOptStatTaskInfo &task_info);
StatTable &stat_table);
static int gather_table_stats_with_default_param(ObExecContext &ctx,
const int64_t duration_time,

View File

@ -911,6 +911,115 @@ int ObBasicStatsEstimator::get_all_tablet_id_and_object_id(const ObTableStatPara
return ret;
}
int ObBasicStatsEstimator::get_need_stats_table_cnt(ObExecContext &ctx,
const int64_t tenant_id,
int64_t &task_table_count)
{
int ret = OB_SUCCESS;
ObSqlString select_sql;
if (OB_FAIL(select_sql.append_fmt(
"select count(1) as cnt from (select m.table_id from " \
"%s m left join %s up on m.table_id = up.table_id and up.pname = 'STALE_PERCENT' join %s gp on gp.sname = 'STALE_PERCENT' " \
"where (case when (m.inserts+m.updates+m.deletes) = 0 then 0 "
"else ((m.inserts+m.updates+m.deletes) - (m.last_inserts+m.last_updates+m.last_deletes)) * 1.0 / (m.inserts+m.updates+m.deletes) > " \
"(CASE WHEN up.valchar IS NOT NULL THEN cast(up.valchar as signed) * 1.0 / 100 ELSE Cast(gp.spare4 AS signed) * 1.0 / 100 end) end)) ",
share::OB_ALL_MONITOR_MODIFIED_TNAME,
share::OB_ALL_OPTSTAT_USER_PREFS_TNAME,
share::OB_ALL_OPTSTAT_GLOBAL_PREFS_TNAME))) {
LOG_WARN("failed to append fmt", K(ret));
} else {
ObCommonSqlProxy *sql_proxy = ctx.get_sql_proxy();
SMART_VAR(ObMySQLProxy::MySQLResult, proxy_result) {
sqlclient::ObMySQLResult *client_result = NULL;
ObSQLClientRetryWeak sql_client_retry_weak(sql_proxy);
if (OB_FAIL(sql_client_retry_weak.read(proxy_result, tenant_id, select_sql.ptr()))) {
LOG_WARN("failed to execute sql", K(ret), K(select_sql));
} else if (OB_ISNULL(client_result = proxy_result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to execute sql", K(ret));
} else {
while (OB_SUCC(ret) && OB_SUCC(client_result->next())) {
int64_t idx = 0;
ObObj obj;
if (OB_FAIL(client_result->get_obj(idx, obj))) {
LOG_WARN("failed to get object", K(ret));
} else if (OB_FAIL(obj.get_int(task_table_count))) {
LOG_WARN("failed to get int", K(ret), K(obj));
}
}
ret = OB_ITER_END == ret ? OB_SUCCESS : ret;
}
int tmp_ret = OB_SUCCESS;
if (NULL != client_result) {
if (OB_SUCCESS != (tmp_ret = client_result->close())) {
LOG_WARN("close result set failed", K(ret), K(tmp_ret));
ret = COVER_SUCC(tmp_ret);
}
}
}
LOG_TRACE("succeed to get table count that need gathering table stats", K(ret), K(task_table_count));
}
return ret;
}
int ObBasicStatsEstimator::get_need_stats_tables(ObExecContext &ctx,
const int64_t tenant_id,
ObIArray<int64_t> &table_ids,
int64_t &slice_cnt)
{
int ret = OB_SUCCESS;
ObSqlString select_sql;
if (OB_FAIL(select_sql.append_fmt(
"select distinct table_id from (select m.table_id from " \
"%s m left join %s up on m.table_id = up.table_id and up.pname = 'STALE_PERCENT' join %s gp on gp.sname = 'STALE_PERCENT' " \
"where (case when (m.inserts+m.updates+m.deletes) = 0 then 0 "
"else ((m.inserts+m.updates+m.deletes) - (m.last_inserts+m.last_updates+m.last_deletes)) * 1.0 / (m.inserts+m.updates+m.deletes) > " \
"(CASE WHEN up.valchar IS NOT NULL THEN cast(up.valchar as signed) * 1.0 / 100 ELSE Cast(gp.spare4 AS signed) * 1.0 / 100 end) end)) "
"ORDER BY table_id DESC limit %ld",
share::OB_ALL_MONITOR_MODIFIED_TNAME,
share::OB_ALL_OPTSTAT_USER_PREFS_TNAME,
share::OB_ALL_OPTSTAT_GLOBAL_PREFS_TNAME,
slice_cnt))) {
LOG_WARN("failed to append fmt", K(ret));
} else {
ObCommonSqlProxy *sql_proxy = ctx.get_sql_proxy();
SMART_VAR(ObMySQLProxy::MySQLResult, proxy_result) {
sqlclient::ObMySQLResult *client_result = NULL;
ObSQLClientRetryWeak sql_client_retry_weak(sql_proxy);
if (OB_FAIL(sql_client_retry_weak.read(proxy_result, tenant_id, select_sql.ptr()))) {
LOG_WARN("failed to execute sql", K(ret), K(select_sql));
} else if (OB_ISNULL(client_result = proxy_result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to execute sql", K(ret));
} else {
while (OB_SUCC(ret) && OB_SUCC(client_result->next())) {
int64_t idx = 0;
ObObj obj;
int64_t table_id = -1;
if (OB_FAIL(client_result->get_obj(idx, obj))) {
LOG_WARN("failed to get object", K(ret));
} else if (OB_FAIL(obj.get_int(table_id))) {
LOG_WARN("failed to get int", K(ret), K(obj));
} else if (OB_FAIL(table_ids.push_back(table_id))) {
LOG_WARN("failed to push back table id", K(ret));
}
}
ret = OB_ITER_END == ret ? OB_SUCCESS : ret;
}
int tmp_ret = OB_SUCCESS;
if (NULL != client_result) {
if (OB_SUCCESS != (tmp_ret = client_result->close())) {
LOG_WARN("close result set failed", K(ret), K(tmp_ret));
ret = COVER_SUCC(tmp_ret);
}
}
}
LOG_TRACE("succeed to get table ids that need gathering table stats",
K(ret), K(slice_cnt), K(tenant_id), K(table_ids.count()), K(table_ids));
}
return ret;
}
int ObBasicStatsEstimator::generate_first_part_idx_map(const ObIArray<PartInfo> &all_part_infos,
hash::ObHashMap<int64_t, int64_t> &first_part_idx_map)
{

View File

@ -109,6 +109,15 @@ public:
ObIArray<ObTabletID> &tablet_ids,
ObIArray<ObObjectID> &partition_ids);
static int get_need_stats_tables(ObExecContext &ctx,
const int64_t tenant_id,
ObIArray<int64_t> &table_ids,
int64_t &slice_cnt);
static int get_need_stats_table_cnt(ObExecContext &ctx,
const int64_t tenant_id,
int64_t &task_table_count);
int estimate(const ObTableStatParam &param,
const ObExtraParam &extra,
ObIArray<ObOptStat> &dst_opt_stats);

View File

@ -89,25 +89,6 @@ int StatTable::assign(const StatTable &other)
return no_regather_partition_ids_.assign(other.no_regather_partition_ids_);
}
int ObGatherTableStatsHelper::get_duration_time(sql::ParamStore &params)
{
int ret = OB_SUCCESS;
duration_time_ = -1;
number::ObNumber num_duration;
if (OB_UNLIKELY(params.empty() || params.at(0).is_null())) {
// do nothing
} else if (lib::is_oracle_mode()) {
if (OB_FAIL(params.at(0).get_number(num_duration))) {
LOG_WARN("failed to get duration", K(ret), K(params.at(0)));
} else if (OB_FAIL(num_duration.extract_valid_int64_with_trunc(duration_time_))) {
LOG_WARN("extract_valid_int64_with_trunc failed", K(ret), K(num_duration));
}
} else if (OB_FAIL(params.at(0).get_int(duration_time_))) {
LOG_WARN("failed to get duration", K(ret), K(params.at(0)));
}
return ret;
}
/**
* @brief
* The order to gather tables

View File

@ -231,30 +231,6 @@ struct ObStatTableWrapper {
K_(last_gather_duration));
};
struct ObGatherTableStatsHelper {
ObGatherTableStatsHelper():
stat_tables_(),
duration_time_(-1),
succeed_count_(0),
failed_count_(0)
{}
inline bool need_gather_table_stats() const
{
return !stat_tables_.empty();
}
int get_duration_time(sql::ParamStore &params);
ObArray<ObStatTableWrapper> stat_tables_;
//duration_time to is used to mark the gather database stats job can use max time. default value
//is -1, it's meaning gather until all table have been gathered.
int64_t duration_time_;
int64_t succeed_count_;
int64_t failed_count_;
TO_STRING_KV(K_(stat_tables),
K_(duration_time),
K_(succeed_count),
K_(failed_count));
};
struct ObGlobalStatParam
{
ObGlobalStatParam()