[CP]optimize gather database stats job proc
This commit is contained in:
parent
c90faddb7b
commit
e8f28b7c7b
@ -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,
|
||||
|
@ -437,19 +437,32 @@ public:
|
||||
sql::ParamStore ¶ms,
|
||||
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,
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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 ¶m,
|
||||
const ObExtraParam &extra,
|
||||
ObIArray<ObOptStat> &dst_opt_stats);
|
||||
|
@ -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 ¶ms)
|
||||
{
|
||||
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
|
||||
|
@ -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 ¶ms);
|
||||
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()
|
||||
|
Loading…
x
Reference in New Issue
Block a user