Fix the core caused by plan cache double free
This commit is contained in:
@ -76,6 +76,7 @@ int ObLCObjectManager::alloc(ObCacheObjGuard& guard,
|
||||
if (OB_FAIL(alloc_cache_obj_map_.set_refactored(obj_id, cache_obj))) {
|
||||
LOG_WARN("failed to add element to hashmap", K(ret));
|
||||
inner_free(cache_obj);
|
||||
entity = NULL;
|
||||
cache_obj = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
@ -54,13 +54,13 @@ int ObPCVSet::init(ObILibCacheCtx &ctx, const ObILibCacheObject *obj)
|
||||
} else {
|
||||
is_inited_ = true;
|
||||
if (pc_ctx.exec_ctx_.get_min_cluster_version() != GET_MIN_CLUSTER_VERSION()) {
|
||||
LOG_DEBUG("Lob Debug, using remote min cluster version",
|
||||
LOG_TRACE("Lob Debug, using remote min cluster version",
|
||||
K(pc_ctx.exec_ctx_.get_min_cluster_version()),
|
||||
K(GET_MIN_CLUSTER_VERSION()));
|
||||
}
|
||||
min_cluster_version_ = pc_ctx.exec_ctx_.get_min_cluster_version();
|
||||
normal_parse_const_cnt_ = pc_ctx.normal_parse_const_cnt_;
|
||||
LOG_DEBUG("inited pcv set", K(pc_key_), K(ObTimeUtility::current_time()));
|
||||
LOG_TRACE("inited pcv set", K(pc_key_), K(ObTimeUtility::current_time()));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -121,14 +121,14 @@ int ObPCVSet::inner_get_cache_obj(ObILibCacheCtx &ctx,
|
||||
} else {
|
||||
if (normal_parse_const_cnt_ != pc_ctx.fp_result_.raw_params_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_PC_LOG(DEBUG, "const number of fast parse and normal parse is different",
|
||||
SQL_PC_LOG(TRACE, "const number of fast parse and normal parse is different",
|
||||
"fast_parse_const_num", pc_ctx.fp_result_.raw_params_.count(),
|
||||
K_(normal_parse_const_cnt),
|
||||
K(pc_ctx.fp_result_.raw_params_));
|
||||
}
|
||||
}
|
||||
if (pc_ctx.exec_ctx_.get_min_cluster_version() != GET_MIN_CLUSTER_VERSION()) {
|
||||
LOG_DEBUG("Lob Debug, using remote min cluster version",
|
||||
LOG_TRACE("Lob Debug, using remote min cluster version",
|
||||
K(pc_ctx.exec_ctx_.get_min_cluster_version()),
|
||||
K(GET_MIN_CLUSTER_VERSION()));
|
||||
}
|
||||
@ -162,26 +162,26 @@ int ObPCVSet::inner_get_cache_obj(ObILibCacheCtx &ctx,
|
||||
bool need_check_schema = true;
|
||||
DLIST_FOREACH(pcv, pcv_list_) {
|
||||
bool is_same = false;
|
||||
LOG_DEBUG("get plan, pcv", K(pcv));
|
||||
LOG_TRACE("get plan, pcv", K(pcv));
|
||||
if (OB_FAIL(pcv->get_all_dep_schema(pc_ctx,
|
||||
pc_ctx.sql_ctx_.session_info_->get_database_id(),
|
||||
new_tenant_schema_version,
|
||||
need_check_schema,
|
||||
schema_array))) {
|
||||
if (OB_OLD_SCHEMA_VERSION == ret) {
|
||||
LOG_DEBUG("failed to get all table schema", K(ret));
|
||||
LOG_TRACE("failed to get all table schema", K(ret));
|
||||
} else {
|
||||
LOG_WARN("failed to get all table schema", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(pcv->match(pc_ctx, schema_array, is_same))) {
|
||||
LOG_WARN("fail to match pcv when get plan", K(ret));
|
||||
} else if (false == is_same) {
|
||||
LOG_DEBUG("failed to match param");
|
||||
LOG_TRACE("failed to match param");
|
||||
/*do nothing*/
|
||||
} else {
|
||||
matched_pcv = pcv;
|
||||
if (OB_FAIL(pcv->choose_plan(pc_ctx, schema_array, plan))) {
|
||||
LOG_DEBUG("failed to get plan in plan cache value", K(ret));
|
||||
LOG_TRACE("failed to get plan in plan cache value", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
@ -248,7 +248,7 @@ int ObPCVSet::inner_add_cache_obj(ObILibCacheCtx &ctx,
|
||||
if (OB_FAIL(pcv->add_plan(*plan, schema_array, pc_ctx))) {
|
||||
if (OB_SQL_PC_PLAN_DUPLICATE == ret
|
||||
|| is_not_supported_err(ret)) {
|
||||
LOG_DEBUG("fail to add plan to pcv", K(ret));
|
||||
LOG_TRACE("fail to add plan to pcv", K(ret));
|
||||
} else {
|
||||
LOG_WARN("fail to add plan to pcv", K(ret));
|
||||
}
|
||||
@ -340,7 +340,7 @@ int ObPCVSet::deep_copy_sql(const ObString &sql)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(sql.ptr())) {
|
||||
SQL_PC_LOG(DEBUG, "sql is empty, ignore copy sql", K(ret), K(lbt()));
|
||||
SQL_PC_LOG(TRACE, "sql is empty, ignore copy sql", K(ret), K(lbt()));
|
||||
} else if (OB_FAIL(ob_write_string(allocator_, sql, sql_))) {
|
||||
SQL_PC_LOG(WARN, "deep copy sql into pcv_set failed", K(ret), K(sql));
|
||||
}
|
||||
@ -493,7 +493,7 @@ int ObPCVSet::check_raw_param_for_dup_col(ObPlanCacheCtx &pc_ctx, bool &contain_
|
||||
|
||||
if (0 != l_tmp_str.compare(r_tmp_str)) {
|
||||
all_same = false;
|
||||
LOG_DEBUG("raw text not matched", K(l_tmp_str), K(r_tmp_str));
|
||||
LOG_TRACE("raw text not matched", K(l_tmp_str), K(r_tmp_str));
|
||||
}
|
||||
}
|
||||
} // for end
|
||||
|
||||
@ -469,7 +469,7 @@ int ObPlanCache::get_plan(common::ObIAllocator &allocator,
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(get_plan_cache(pc_ctx, guard))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to get plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to get plan", K(ret), K(pc_ctx.fp_result_.pc_key_));
|
||||
} else if (OB_ISNULL(guard.cache_obj_)
|
||||
|| ObLibCacheNameSpace::NS_CRSR != guard.cache_obj_->get_ns()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -671,11 +671,11 @@ int ObPlanCache::get_plan_cache(ObILibCacheCtx &ctx,
|
||||
ObPlanCacheCtx &pc_ctx = static_cast<ObPlanCacheCtx&>(ctx);
|
||||
pc_ctx.key_ = &(pc_ctx.fp_result_.pc_key_);
|
||||
if (OB_FAIL(get_cache_obj(ctx, pc_ctx.key_, guard))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to get plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to get plan", K(ret), K(pc_ctx.key_));
|
||||
}
|
||||
// check the returned error code and whether the plan has expired
|
||||
if (OB_FAIL(check_after_get_plan(ret, ctx, guard.cache_obj_))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to check after get plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to check after get plan", K(ret));
|
||||
}
|
||||
if (OB_FAIL(ret) && OB_NOT_NULL(guard.cache_obj_)) {
|
||||
co_mgr_.free(guard.cache_obj_, guard.ref_handle_);
|
||||
@ -699,7 +699,7 @@ int ObPlanCache::add_cache_obj(ObILibCacheCtx &ctx,
|
||||
SQL_PC_LOG(ERROR, "unmatched tenant_id", K(ret), K(get_tenant_id()), K(cache_obj->get_tenant_id()));
|
||||
} else if (OB_FAIL(get_value(key, cache_node, w_ref_lock /*write locked*/))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_PC_LOG(DEBUG, "failed to get cache node from lib cache by key", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to get cache node from lib cache by key", K(ret));
|
||||
} else if (NULL == cache_node) {
|
||||
ObILibCacheKey *cache_key = NULL;
|
||||
//create node, init node, and add cache obj
|
||||
@ -730,7 +730,7 @@ int ObPlanCache::add_cache_obj(ObILibCacheCtx &ctx,
|
||||
SQL_PC_LOG(TRACE, "fail to add cache obj", K(ret), K(cache_obj));
|
||||
}
|
||||
} else if (OB_SUCCESS == hash_err) {
|
||||
SQL_PC_LOG(DEBUG, "succeed to set node to key_node_map");
|
||||
SQL_PC_LOG(TRACE, "succeed to set node to key_node_map");
|
||||
/* stat must be added after set_refactored is successful, otherwise the following may occur:
|
||||
* Thread A Thread B
|
||||
* create_node_and_add_cache_obj
|
||||
@ -780,7 +780,7 @@ int ObPlanCache::add_cache_obj(ObILibCacheCtx &ctx,
|
||||
} else { /* node exist, add cache obj to it */
|
||||
LOG_TRACE("inner add cache obj", K(key), K(cache_node));
|
||||
if (OB_FAIL(cache_node->add_cache_obj(ctx, key, cache_obj))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to add cache obj to lib cache node", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to add cache obj to lib cache node", K(ret));
|
||||
} else if (OB_FAIL(cache_node->update_node_stat(ctx))) {
|
||||
SQL_PC_LOG(WARN, "failed to update node stat", K(ret));
|
||||
} else if (OB_FAIL(add_stat_for_cache_obj(ctx, cache_obj))) {
|
||||
@ -807,21 +807,21 @@ int ObPlanCache::get_cache_obj(ObILibCacheCtx &ctx,
|
||||
SQL_PC_LOG(WARN, "invalid null argument", K(ret), K(key));
|
||||
} else if (OB_FAIL(get_value(key, cache_node, r_ref_lock /*read locked*/))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_PC_LOG(DEBUG, "failed to get cache node from lib cache by key", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to get cache node from lib cache by key", K(ret));
|
||||
} else if (OB_UNLIKELY(NULL == cache_node)) {
|
||||
ret = OB_SQL_PC_NOT_EXIST;
|
||||
SQL_PC_LOG(DEBUG, "cache obj does not exist!", K(key));
|
||||
SQL_PC_LOG(TRACE, "cache obj does not exist!", K(key));
|
||||
} else {
|
||||
LOG_DEBUG("inner_get_cache_obj", K(key), K(cache_node));
|
||||
LOG_TRACE("inner_get_cache_obj", K(key), K(cache_node));
|
||||
if (OB_FAIL(cache_node->update_node_stat(ctx))) {
|
||||
SQL_PC_LOG(WARN, "failed to update node stat", K(ret));
|
||||
} else if (OB_FAIL(cache_node->get_cache_obj(ctx, key, cache_obj))) {
|
||||
if (OB_SQL_PC_NOT_EXIST != ret) {
|
||||
LOG_DEBUG("cache_node fail to get cache obj", K(ret));
|
||||
LOG_TRACE("cache_node fail to get cache obj", K(ret));
|
||||
}
|
||||
} else {
|
||||
guard.cache_obj_ = cache_obj;
|
||||
LOG_DEBUG("succ to get cache obj", KPC(key));
|
||||
LOG_TRACE("succ to get cache obj", KPC(key));
|
||||
}
|
||||
// release lock whatever
|
||||
(void)cache_node->unlock();
|
||||
@ -845,7 +845,7 @@ int ObPlanCache::cache_node_exists(ObILibCacheKey* key,
|
||||
SQL_PC_LOG(WARN, "invalid null argument", K(ret), K(key));
|
||||
} else if (OB_FAIL(get_value(key, cache_node, r_ref_lock /*read locked*/))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_PC_LOG(DEBUG, "failed to get cache node from lib cache by key", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to get cache node from lib cache by key", K(ret));
|
||||
} else if (OB_UNLIKELY(NULL == cache_node)) {
|
||||
is_exists = false;
|
||||
} else {
|
||||
@ -869,12 +869,12 @@ int ObPlanCache::get_value(ObILibCacheKey *key,
|
||||
case OB_SUCCESS: {
|
||||
//get node and lock
|
||||
if (OB_FAIL(op.get_value(node))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to lock cache node", K(ret), KPC(key));
|
||||
SQL_PC_LOG(TRACE, "failed to lock cache node", K(ret), KPC(key));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OB_HASH_NOT_EXIST: { //返回时 node = NULL; ret = OB_SUCCESS;
|
||||
SQL_PC_LOG(DEBUG, "entry does not exist.", KPC(key));
|
||||
SQL_PC_LOG(TRACE, "entry does not exist.", KPC(key));
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
@ -910,6 +910,7 @@ int ObPlanCache::foreach_cache_evict(CallBack &cb)
|
||||
} else if (OB_FAIL(batch_remove_cache_node(*to_evict_list))) {
|
||||
SQL_PC_LOG(WARN, "failed to remove lib cache node", K(ret));
|
||||
}
|
||||
if (OB_NOT_NULL(to_evict_list)) {
|
||||
//decrement reference count
|
||||
int64_t N = to_evict_list->count();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) {
|
||||
@ -917,6 +918,7 @@ int ObPlanCache::foreach_cache_evict(CallBack &cb)
|
||||
to_evict_list->at(i).node_->dec_ref_count(cb.get_ref_handle());
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -926,26 +928,26 @@ template int ObPlanCache::foreach_cache_evict<pl::ObGetPLKVEntryOp>(pl::ObGetPLK
|
||||
int ObPlanCache::cache_evict_all_obj()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SQL_PC_LOG(DEBUG, "cache evict all plan start");
|
||||
SQL_PC_LOG(TRACE, "cache evict all plan start");
|
||||
LCKeyValueArray to_evict_keys;
|
||||
ObKVEntryTraverseOp get_ids_op(&to_evict_keys, PCV_GET_PLAN_KEY_HANDLE);
|
||||
if (OB_FAIL(foreach_cache_evict(get_ids_op))) {
|
||||
SQL_PC_LOG(WARN, "failed to foreach cache evict", K(ret));
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "cache evict all plan end");
|
||||
SQL_PC_LOG(TRACE, "cache evict all plan end");
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPlanCache::cache_evict_by_ns(ObLibCacheNameSpace ns)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SQL_PC_LOG(DEBUG, "cache evict all obj by ns start");
|
||||
SQL_PC_LOG(TRACE, "cache evict all obj by ns start");
|
||||
LCKeyValueArray to_evict_keys;
|
||||
ObGetKVEntryByNsOp get_ids_op(ns, &to_evict_keys, PCV_GET_PLAN_KEY_HANDLE);
|
||||
if (OB_FAIL(foreach_cache_evict(get_ids_op))) {
|
||||
SQL_PC_LOG(WARN, "failed to foreach cache evict", K(ret));
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "cache evict all obj by ns end");
|
||||
SQL_PC_LOG(TRACE, "cache evict all obj by ns end");
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -963,13 +965,13 @@ int ObPlanCache::cache_evict_all_plan()
|
||||
int ObPlanCache::cache_evict_plan_by_sql_id(uint64_t db_id, common::ObString sql_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
SQL_PC_LOG(DEBUG, "cache evict plan by sql id start");
|
||||
SQL_PC_LOG(TRACE, "cache evict plan by sql id start");
|
||||
LCKeyValueArray to_evict_keys;
|
||||
ObGetKVEntryBySQLIDOp get_ids_op(db_id, sql_id, &to_evict_keys, PCV_GET_PLAN_KEY_HANDLE);
|
||||
if (OB_FAIL(foreach_cache_evict(get_ids_op))) {
|
||||
SQL_PC_LOG(WARN, "failed to foreach cache evict", K(ret));
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "cache evict plan by sql id end");
|
||||
SQL_PC_LOG(TRACE, "cache evict plan by sql id end");
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -979,13 +981,13 @@ int ObPlanCache::evict_plan_by_table_name(uint64_t database_id, ObString tab_nam
|
||||
int ret = OB_SUCCESS;
|
||||
observer::ObReqTimeGuard req_timeinfo_guard;
|
||||
ObGlobalReqTimeService::check_req_timeinfo();
|
||||
SQL_PC_LOG(DEBUG, "cache evict plan by table name start");
|
||||
SQL_PC_LOG(TRACE, "cache evict plan by table name start");
|
||||
LCKeyValueArray to_evict_keys;
|
||||
ObGetPcvSetByTabNameOp get_ids_op(database_id, tab_name, &to_evict_keys, PCV_GET_PLAN_KEY_HANDLE);
|
||||
if (OB_FAIL(foreach_cache_evict(get_ids_op))) {
|
||||
SQL_PC_LOG(WARN, "failed to foreach cache evict", K(ret));
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "cache evict plan baseline by sql id end");
|
||||
SQL_PC_LOG(TRACE, "cache evict plan baseline by sql id end");
|
||||
|
||||
return ret;
|
||||
}
|
||||
@ -1253,7 +1255,7 @@ int ObPlanCache::add_cache_obj_stat(ObILibCacheCtx &ctx, ObILibCacheObject *cach
|
||||
co_mgr_.free(cache_obj, LC_REF_CACHE_OBJ_STAT_HANDLE);
|
||||
cache_obj = NULL;
|
||||
} else {
|
||||
LOG_DEBUG("succeeded to add cache object stat", K(cache_obj->get_object_id()), K(cache_obj));
|
||||
LOG_TRACE("succeeded to add cache object stat", K(cache_obj->get_object_id()), K(cache_obj));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -1453,7 +1455,7 @@ int ObPlanCache::add_ps_plan(T *plan, ObPlanCacheCtx &pc_ctx)
|
||||
SQL_PC_LOG(WARN, "invalid physical plan", K(ret));
|
||||
} else if (is_reach_memory_limit()) {
|
||||
ret = OB_REACH_MEMORY_LIMIT;
|
||||
SQL_PC_LOG(DEBUG, "plan cache memory used reach the high water mark",
|
||||
SQL_PC_LOG(TRACE, "plan cache memory used reach the high water mark",
|
||||
K(mem_used_), K(get_mem_limit()), K(ret));
|
||||
} else if (plan->get_mem_size() >= get_mem_high()) {
|
||||
// plan mem is too big to reach memory highwater, do not add plan
|
||||
@ -1493,7 +1495,7 @@ int ObPlanCache::add_exists_cache_obj_by_sql(ObILibCacheCtx &ctx,
|
||||
pc_ctx.need_add_obj_stat_ = false;
|
||||
uint64_t old_stmt_id = pc_ctx.fp_result_.pc_key_.key_id_;
|
||||
pc_ctx.fp_result_.pc_key_.key_id_ = OB_INVALID_ID;
|
||||
SQL_PC_LOG(DEBUG, "start to add ps plan by sql", K(pc_ctx.fp_result_.pc_key_));
|
||||
SQL_PC_LOG(TRACE, "start to add ps plan by sql", K(pc_ctx.fp_result_.pc_key_));
|
||||
if (OB_FAIL(add_plan_cache(pc_ctx, cache_obj))) {
|
||||
if (OB_FAIL(deal_add_ps_plan_result(ret, pc_ctx, *cache_obj))) {
|
||||
LOG_WARN("fail to deal result code", K(ret));
|
||||
@ -1515,7 +1517,7 @@ int ObPlanCache::deal_add_ps_plan_result(int add_plan_ret,
|
||||
int ret = add_plan_ret;
|
||||
if (OB_SQL_PC_PLAN_DUPLICATE == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
LOG_DEBUG("this plan has been added by others, need not add again", K(cache_object));
|
||||
LOG_TRACE("this plan has been added by others, need not add again", K(cache_object));
|
||||
} else if (OB_REACH_MEMORY_LIMIT == ret || OB_SQL_PC_PLAN_SIZE_LIMIT == ret) {
|
||||
if (REACH_TIME_INTERVAL(1000000)) { //1s, 当内存达到上限时, 该日志打印会比较频繁, 所以以1s为间隔打印
|
||||
ObTruncatedString trunc_sql(pc_ctx.raw_sql_);
|
||||
@ -1526,7 +1528,7 @@ int ObPlanCache::deal_add_ps_plan_result(int add_plan_ret,
|
||||
ret = OB_SUCCESS;
|
||||
} else if (is_not_supported_err(ret)) {
|
||||
ret = OB_SUCCESS;
|
||||
LOG_DEBUG("plan cache don't support add this kind of plan now", K(cache_object));
|
||||
LOG_TRACE("plan cache don't support add this kind of plan now", K(cache_object));
|
||||
} else if (OB_FAIL(ret)) {
|
||||
if (OB_REACH_MAX_CONCURRENT_NUM != ret) { //如果是达到限流上限, 则将错误码抛出去
|
||||
ret = OB_SUCCESS; //add plan出错, 覆盖错误码, 确保因plan cache失败不影响正常执行路径
|
||||
@ -1534,7 +1536,7 @@ int ObPlanCache::deal_add_ps_plan_result(int add_plan_ret,
|
||||
}
|
||||
} else {
|
||||
pc_ctx.sql_ctx_.self_add_plan_ = true;
|
||||
LOG_DEBUG("Successed to add plan to ObPlanCache", K(cache_object));
|
||||
LOG_TRACE("Successed to add plan to ObPlanCache", K(cache_object));
|
||||
}
|
||||
|
||||
return ret;
|
||||
@ -1559,7 +1561,7 @@ int ObPlanCache::add_exists_cache_obj_by_stmt_id(ObILibCacheCtx &ctx,
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_PC_LOG(WARN, "convert cache_obj to ObPhysicalPlan failed", K(ret));
|
||||
} else {
|
||||
SQL_PC_LOG(DEBUG, "ps_stmt_id changed", K(plan->stat_.ps_stmt_id_), K(new_stmt_id));
|
||||
SQL_PC_LOG(TRACE, "ps_stmt_id changed", K(plan->stat_.ps_stmt_id_), K(new_stmt_id));
|
||||
plan->stat_.ps_stmt_id_ = new_stmt_id;
|
||||
}
|
||||
} else {
|
||||
@ -1708,13 +1710,13 @@ int ObPlanCache::get_ps_plan(ObCacheObjGuard& guard,
|
||||
pc_ctx.fp_result_.pc_key_.name_ = pc_ctx.raw_sql_;
|
||||
if (OB_FAIL(get_plan_cache(pc_ctx, guard))) {
|
||||
if (OB_OLD_SCHEMA_VERSION == ret) {
|
||||
SQL_PC_LOG(DEBUG, "get cache obj by sql failed because of old_schema_version",
|
||||
SQL_PC_LOG(TRACE, "get cache obj by sql failed because of old_schema_version",
|
||||
K(ret), K(pc_ctx.raw_sql_), K(new_stmt_id), K(cache_key_node_map_.size()));
|
||||
} else {
|
||||
SQL_PC_LOG(DEBUG, "get cache obj by sql failed",
|
||||
SQL_PC_LOG(TRACE, "get cache obj by sql failed",
|
||||
K(ret), K(pc_ctx.raw_sql_), K(new_stmt_id), K(cache_key_node_map_.size()));
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "fail to get plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "fail to get plan", K(ret));
|
||||
} else if (OB_ISNULL(guard.cache_obj_) || OB_UNLIKELY(!guard.cache_obj_->is_valid_cache_obj())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cache obj is invalid", K(ret), KPC(guard.cache_obj_));
|
||||
@ -1797,7 +1799,7 @@ int ObPlanCache::need_late_compile(ObPhysicalPlan *plan,
|
||||
need_late_compilation = false;
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("will use late compilation", K(need_late_compilation));
|
||||
LOG_TRACE("will use late compilation", K(need_late_compilation));
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
@ -452,7 +452,7 @@ int ObPlanCacheValue::choose_plan(ObPlanCacheCtx &pc_ctx,
|
||||
SQL_PC_LOG(WARN, "fail to check table version", K(ret));
|
||||
} else if (true == is_old_version) {
|
||||
ret = OB_OLD_SCHEMA_VERSION;
|
||||
SQL_PC_LOG(DEBUG, "view or table is old version", K(ret));
|
||||
SQL_PC_LOG(TRACE, "view or table is old version", K(ret));
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
//do nothing
|
||||
@ -703,28 +703,28 @@ int ObPlanCacheValue::resolver_params(ObPlanCacheCtx &pc_ctx,
|
||||
&& (value.is_negative_number()
|
||||
|| (value.is_zero_number() && '-' == raw_param->str_value_[0]))) { // -0 is also counted as negative
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_DEBUG("param must be positive", K(ret), K(i), K(value));
|
||||
LOG_TRACE("param must be positive", K(ret), K(i), K(value));
|
||||
pc_ctx.should_add_plan_ = false; // 内部主动抛出not supported时候需要设置这个标志,以免新计划add plan导致锁冲突
|
||||
} else if (lib::is_mysql_mode()
|
||||
&& value.is_integer_type()
|
||||
&& (value.get_int() < 0
|
||||
|| (0 == value.get_int() && '-' == raw_param->str_value_[0]))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_DEBUG("param must be positive", K(ret), K(i), K(value));
|
||||
LOG_TRACE("param must be positive", K(ret), K(i), K(value));
|
||||
pc_ctx.should_add_plan_ = false; // 内部主动抛出not supported时候需要设置这个标志,以免新计划add plan导致锁冲突
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "is_param",
|
||||
SQL_PC_LOG(TRACE, "is_param",
|
||||
K(i),
|
||||
K(value),
|
||||
K(raw_param->type_),
|
||||
K(raw_param->value_),
|
||||
"str_value", ObString(raw_param->str_len_, raw_param->str_value_));
|
||||
} else {
|
||||
SQL_PC_LOG(DEBUG, "not_param",
|
||||
SQL_PC_LOG(TRACE, "not_param",
|
||||
K(i),
|
||||
K(value),
|
||||
K(raw_param->type_),
|
||||
@ -888,11 +888,11 @@ int ObPlanCacheValue::check_not_param_value(const ObFastParserResult &fp_result
|
||||
} else if (0 != not_param_info.at(i).raw_text_.compare(
|
||||
ObString(raw_param->text_len_, raw_param->raw_text_))) {
|
||||
is_same = false;
|
||||
LOG_DEBUG("match not param info",
|
||||
LOG_TRACE("match not param info",
|
||||
"raw value", ObString(raw_param->text_len_, raw_param->raw_text_),
|
||||
"cached special value", not_param_info.at(i).raw_text_);
|
||||
} else {
|
||||
LOG_DEBUG("match param info",
|
||||
LOG_TRACE("match param info",
|
||||
"raw value", ObString(raw_param->text_len_, raw_param->raw_text_),
|
||||
"cached special value", not_param_info.at(i).raw_text_);
|
||||
}
|
||||
@ -914,10 +914,10 @@ int ObPlanCacheValue::cmp_not_param_info(const NotParamInfoList &l_param_info_li
|
||||
const NotParamInfo &r_param_info = r_param_info_list.at(i);
|
||||
if (l_param_info.idx_ != r_param_info.idx_) {
|
||||
is_equal = false;
|
||||
LOG_DEBUG("compare not param info", K(l_param_info), K(r_param_info));
|
||||
LOG_TRACE("compare not param info", K(l_param_info), K(r_param_info));
|
||||
} else if (0 != l_param_info.raw_text_.compare(r_param_info.raw_text_)) {
|
||||
is_equal = false;
|
||||
LOG_DEBUG("compare not param info", K(l_param_info), K(r_param_info));
|
||||
LOG_TRACE("compare not param info", K(l_param_info), K(r_param_info));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1001,7 +1001,7 @@ int ObPlanCacheValue::get_one_group_params(int64_t pos, const ParamStore &src_pa
|
||||
} else if (OB_FAIL(dst_params.push_back(array_obj->data_[pos]))) {
|
||||
LOG_WARN("fail to push param_obj to param_store", K(i), K(pos), K(array_obj->data_[pos]), K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("get one batch obj", K(pos), K(i), K(array_obj->data_[pos]));
|
||||
LOG_TRACE("get one batch obj", K(pos), K(i), K(array_obj->data_[pos]));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -1085,7 +1085,7 @@ int ObPlanCacheValue::add_plan(ObPlanCacheObject &plan,
|
||||
SQL_PC_LOG(WARN, "fail to check table version", K(ret));
|
||||
} else if (is_old_version) {
|
||||
ret = OB_OLD_SCHEMA_VERSION;
|
||||
SQL_PC_LOG(DEBUG, "view or table is old version", K(ret));
|
||||
SQL_PC_LOG(TRACE, "view or table is old version", K(ret));
|
||||
/* Consider this concurrent scene:
|
||||
1. No mapping is defined on t.c1 at first.
|
||||
2. Thread1 resolve select * from t where c1 = 1; and generate a plan with rule_id = INVALID
|
||||
@ -1130,7 +1130,7 @@ int ObPlanCacheValue::add_plan(ObPlanCacheObject &plan,
|
||||
need_new_planset = false;
|
||||
batch_plan_set = cur_plan_set;
|
||||
if (OB_FAIL(cur_plan_set->add_cache_obj(plan, pc_ctx, outline_param_idx, add_plan_ret))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to add plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to add plan", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
@ -1155,7 +1155,7 @@ int ObPlanCacheValue::add_plan(ObPlanCacheObject &plan,
|
||||
get_pc_malloc()))) {
|
||||
LOG_WARN("init new plan set failed", K(ret));
|
||||
} else if (OB_FAIL(plan_set->add_cache_obj(plan, pc_ctx, outline_param_idx, add_plan_ret))) {
|
||||
SQL_PC_LOG(DEBUG, "failed to add plan to plan set", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to add plan to plan set", K(ret));
|
||||
} else if (!plan_sets_.add_last(plan_set)) {
|
||||
ret = OB_ERROR;
|
||||
SQL_PC_LOG(WARN, "failed to add plan set to plan cache value", K(ret));
|
||||
@ -1233,7 +1233,7 @@ void ObPlanCacheValue::reset()
|
||||
plan_sets_.clear();
|
||||
// free plan_cache_key
|
||||
if (NULL == pc_alloc_) {
|
||||
SQL_PC_LOG(DEBUG, "pc alloc not init, may be reset before", K(pc_alloc_));
|
||||
SQL_PC_LOG(TRACE, "pc alloc not init, may be reset before", K(pc_alloc_));
|
||||
} else {
|
||||
for (int64_t i = 0; i < not_param_info_.count(); i++) {
|
||||
if (NULL != not_param_info_.at(i).raw_text_.ptr()) {
|
||||
@ -1410,7 +1410,7 @@ int ObPlanCacheValue::check_dep_schema_version(const ObIArray<PCVSchemaObj> &sch
|
||||
LOG_DEBUG("matched schema objs", K(*schema_obj1), K(schema_obj2), K(i));
|
||||
// do nothing
|
||||
} else {
|
||||
LOG_DEBUG("mismatched schema objs", K(*schema_obj1), K(schema_obj2), K(i));
|
||||
LOG_TRACE("mismatched schema objs", K(*schema_obj1), K(schema_obj2), K(i));
|
||||
is_old_version = true;
|
||||
}
|
||||
}
|
||||
@ -1747,7 +1747,7 @@ int ObPlanCacheValue::set_stored_schema_objs(const DependenyTableStore &dep_tabl
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
LOG_DEBUG("check sys table", K(table_schema->get_table_name()), K(contain_sys_name_table_));
|
||||
LOG_TRACE("check sys table", K(table_schema->get_table_name()), K(contain_sys_name_table_));
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
@ -1881,7 +1881,7 @@ int ObPlanCacheValue::get_all_dep_schema(ObPlanCacheCtx &pc_ctx,
|
||||
tmp_schema_obj.reset();
|
||||
} else if (nullptr == table_schema) {
|
||||
ret = OB_OLD_SCHEMA_VERSION;
|
||||
LOG_DEBUG("table not exist", K(ret), K(*pcv_schema), K(table_schema));
|
||||
LOG_TRACE("table not exist", K(ret), K(*pcv_schema), K(table_schema));
|
||||
} else if (OB_FAIL(tmp_schema_obj.init_without_copy_name(table_schema))) {
|
||||
LOG_WARN("failed to init pcv schema obj", K(ret));
|
||||
} else if (OB_FAIL(schema_array.push_back(tmp_schema_obj))) {
|
||||
|
||||
@ -83,7 +83,7 @@ int ObPlanSet::match_params_info(const ParamStore *params,
|
||||
} else {
|
||||
//匹配原始的参数
|
||||
int64_t N = params->count();
|
||||
LOG_DEBUG("params info", K(params_info_), K(*params), K(this));
|
||||
LOG_TRACE("params info", K(params_info_), K(*params), K(this));
|
||||
for (int64_t i = 0; OB_SUCC(ret) && is_same && i < N; ++i) {
|
||||
if (OB_FAIL(match_param_info(params_info_.at(i),
|
||||
params->at(i),
|
||||
@ -196,6 +196,7 @@ int ObPlanSet::match_params_info(const ParamStore *params,
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
is_same = false;
|
||||
LOG_TRACE("after match param result", K(ret), K(is_same), K(params_info_));
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("after match param result", K(ret), K(is_same), K(params_info_));
|
||||
@ -261,6 +262,7 @@ int ObPlanSet::match_param_info(const ObParamInfo ¶m_info,
|
||||
is_same = true;
|
||||
} else {
|
||||
is_same = false;
|
||||
LOG_TRACE("ext match param info", K(data_type), K(param_info), K(is_same), K(ret));
|
||||
}
|
||||
LOG_DEBUG("ext match param info", K(data_type), K(param_info), K(is_same), K(ret));
|
||||
} else if (param_info.is_oracle_empty_string_ && !param.is_null()) { //普通字符串不匹配空串的计划
|
||||
@ -934,7 +936,7 @@ int ObPlanSet::match_constraint(const ParamStore ¶ms, bool &is_matched)
|
||||
K(ret), K(const_param.get_type()), K(params.at(param_idx).get_type()));
|
||||
} else if (!const_param.can_compare(params.at(param_idx)) ||
|
||||
0 != const_param.compare(params.at(param_idx))) {
|
||||
LOG_DEBUG("not matched const param", K(const_param), K(params.at(param_idx)));
|
||||
LOG_TRACE("not matched const param", K(const_param), K(params.at(param_idx)));
|
||||
is_matched = false;
|
||||
} else {
|
||||
// do nothing
|
||||
@ -968,7 +970,7 @@ int ObPlanSet::match_constraint(const ParamStore ¶ms, bool &is_matched)
|
||||
}
|
||||
}
|
||||
if (match_const) {
|
||||
LOG_DEBUG("matched const param constraint", K(params), K(all_possible_const_param_constraints_.at(i)));
|
||||
LOG_TRACE("matched const param constraint", K(params), K(all_possible_const_param_constraints_.at(i)));
|
||||
is_matched = false; // matching one of the constraint, need to generated new plan
|
||||
}
|
||||
}
|
||||
@ -1016,7 +1018,7 @@ int ObPlanSet::match_constraint(const ParamStore ¶ms, bool &is_matched)
|
||||
}
|
||||
if (OB_SUCC(ret) && !is_matched) {
|
||||
is_matched = false;
|
||||
LOG_DEBUG("not match equal param constraint", K(params), K(first_idx), K(second_idx));
|
||||
LOG_TRACE("not match equal param constraint", K(params), K(first_idx), K(second_idx));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1174,7 +1176,7 @@ int ObSqlPlanSet::add_plan(ObPhysicalPlan &plan,
|
||||
if (OB_SUCC(ret)) {
|
||||
switch(plan_type) {
|
||||
case OB_PHY_PLAN_LOCAL:{
|
||||
SQL_PC_LOG(DEBUG, "plan set add plan, local plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "plan set add plan, local plan", K(ret));
|
||||
if (is_multi_stmt_plan()) {
|
||||
if (NULL != array_binding_plan_) {
|
||||
ret = OB_SQL_PC_PLAN_DUPLICATE;
|
||||
@ -1183,7 +1185,7 @@ int ObSqlPlanSet::add_plan(ObPhysicalPlan &plan,
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(add_physical_plan(OB_PHY_PLAN_LOCAL, pc_ctx, plan))) {
|
||||
SQL_PC_LOG(DEBUG, "fail to add local plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "fail to add local plan", K(ret));
|
||||
} else if (OB_SUCC(ret)
|
||||
&& FALSE_IT(direct_local_plan_ = &plan)) {
|
||||
// do nothing
|
||||
@ -1207,7 +1209,7 @@ int ObSqlPlanSet::add_plan(ObPhysicalPlan &plan,
|
||||
}
|
||||
} break;
|
||||
case OB_PHY_PLAN_DISTRIBUTED: {
|
||||
SQL_PC_LOG(DEBUG, "plan set add plan, distr plan", K(ret));
|
||||
SQL_PC_LOG(TRACE, "plan set add plan, distr plan", K(ret));
|
||||
if (OB_FAIL(add_physical_plan(OB_PHY_PLAN_DISTRIBUTED, pc_ctx, plan))) {
|
||||
LOG_WARN("failed to add dist plan", K(ret), K(plan));
|
||||
} else {
|
||||
@ -1230,7 +1232,7 @@ int ObSqlPlanSet::add_plan(ObPhysicalPlan &plan,
|
||||
}
|
||||
}
|
||||
}
|
||||
SQL_PC_LOG(DEBUG, "plan set add plan", K(ret), K(&plan), "plan type ", plan_type,
|
||||
SQL_PC_LOG(TRACE, "plan set add plan", K(ret), K(&plan), "plan type ", plan_type,
|
||||
K(has_duplicate_table_), K(stmt_type_));
|
||||
// increase plan ref_count,
|
||||
// if plan doesn't add in plan cache,don't increase ref_count;
|
||||
@ -1319,7 +1321,7 @@ int ObSqlPlanSet::init_new_set(const ObPlanCacheCtx &pc_ctx,
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
|
||||
if (NULL == partition_infos.at(i)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_PC_LOG(DEBUG, "invalid partition info");
|
||||
SQL_PC_LOG(TRACE, "invalid partition info");
|
||||
} else if (OB_FAIL(table_locations_.push_back(partition_infos.at(i)->get_table_location()))) {
|
||||
SQL_PC_LOG(WARN, "fail to push table location", K(ret));
|
||||
} else if (is_all_non_partition_
|
||||
@ -1342,7 +1344,7 @@ int ObSqlPlanSet::select_plan(ObPlanCacheCtx &pc_ctx, ObPlanCacheObject *&cache_
|
||||
} else if (0 == need_try_plan_ || is_multi_stmt_plan()) {
|
||||
if (OB_FAIL(get_plan_normal(pc_ctx, plan))) {
|
||||
if (OB_SQL_PC_NOT_EXIST == ret) {
|
||||
LOG_DEBUG("fail to get plan normal", K(ret));
|
||||
LOG_TRACE("fail to get plan normal", K(ret));
|
||||
} else {
|
||||
LOG_WARN("fail to get plan normal", K(ret));
|
||||
}
|
||||
@ -1350,7 +1352,7 @@ int ObSqlPlanSet::select_plan(ObPlanCacheCtx &pc_ctx, ObPlanCacheObject *&cache_
|
||||
} else {
|
||||
if (OB_FAIL(get_plan_special(pc_ctx, plan))) {
|
||||
if (OB_SQL_PC_NOT_EXIST == ret) {
|
||||
LOG_DEBUG("fail to get plan special", K(ret));
|
||||
LOG_TRACE("fail to get plan special", K(ret));
|
||||
} else {
|
||||
LOG_WARN("fail to get plan special", K(ret));
|
||||
}
|
||||
@ -1591,7 +1593,7 @@ int ObSqlPlanSet::get_plan_normal(ObPlanCacheCtx &pc_ctx,
|
||||
SMART_VAR(PLS, candi_table_locs) {
|
||||
if (enable_inner_part_parallel_exec_) {
|
||||
if (OB_FAIL(get_physical_plan(OB_PHY_PLAN_DISTRIBUTED, pc_ctx, plan))) {
|
||||
LOG_DEBUG("failed to get px plan", K(ret));
|
||||
LOG_TRACE("failed to get px plan", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(get_plan_type(table_locations_,
|
||||
false,
|
||||
@ -1599,7 +1601,7 @@ int ObSqlPlanSet::get_plan_normal(ObPlanCacheCtx &pc_ctx,
|
||||
candi_table_locs,
|
||||
plan_type))) {
|
||||
// ret = OB_SQL_PC_NOT_EXIST;
|
||||
SQL_PC_LOG(DEBUG, "failed to get plan type", K(ret));
|
||||
SQL_PC_LOG(TRACE, "failed to get plan type", K(ret));
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && !enable_inner_part_parallel_exec_) {
|
||||
@ -1613,7 +1615,7 @@ int ObSqlPlanSet::get_plan_normal(ObPlanCacheCtx &pc_ctx,
|
||||
plan = array_binding_plan_;
|
||||
}
|
||||
} else if (OB_FAIL(get_physical_plan(OB_PHY_PLAN_LOCAL, pc_ctx, plan))) {
|
||||
LOG_DEBUG("failed to get local plan", K(ret));
|
||||
LOG_TRACE("failed to get local plan", K(ret));
|
||||
}
|
||||
} break;
|
||||
case OB_PHY_PLAN_REMOTE: {
|
||||
@ -1625,7 +1627,7 @@ int ObSqlPlanSet::get_plan_normal(ObPlanCacheCtx &pc_ctx,
|
||||
case OB_PHY_PLAN_DISTRIBUTED: {
|
||||
if (OB_FAIL(get_physical_plan(OB_PHY_PLAN_DISTRIBUTED, pc_ctx, plan))) {
|
||||
if (OB_SQL_PC_NOT_EXIST == ret) {
|
||||
LOG_DEBUG("fail to get dist plan", K(ret));
|
||||
LOG_TRACE("fail to get dist plan", K(ret));
|
||||
} else {
|
||||
LOG_WARN("fail to get dist plan", K(ret));
|
||||
}
|
||||
@ -1713,9 +1715,9 @@ int ObSqlPlanSet::try_get_dist_plan(ObPlanCacheCtx &pc_ctx,
|
||||
int ret = OB_SUCCESS;
|
||||
plan = NULL;
|
||||
if (OB_FAIL(dist_plans_.get_plan(pc_ctx, plan))) {
|
||||
LOG_DEBUG("failed to get dist plan", K(ret));
|
||||
LOG_TRACE("failed to get dist plan", K(ret));
|
||||
} else if (plan != NULL) {
|
||||
LOG_DEBUG("succeed to get dist plan", K(*plan));
|
||||
LOG_TRACE("succeed to get dist plan", K(*plan));
|
||||
}
|
||||
if (OB_SQL_PC_NOT_EXIST == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
@ -1748,7 +1750,7 @@ int ObSqlPlanSet::get_plan_special(ObPlanCacheCtx &pc_ctx,
|
||||
//try dist plan
|
||||
if (OB_SUCC(ret) && get_next) {
|
||||
if (OB_FAIL(try_get_dist_plan(pc_ctx, plan))) {
|
||||
LOG_DEBUG("failed to try get dist plan", K(ret));
|
||||
LOG_TRACE("failed to try get dist plan", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && nullptr == plan) {
|
||||
|
||||
Reference in New Issue
Block a user