diff --git a/deps/oblib/src/lib/hash/ob_dchash.h b/deps/oblib/src/lib/hash/ob_dchash.h index 31291f7d23..bb0f3695c6 100644 --- a/deps/oblib/src/lib/hash/ob_dchash.h +++ b/deps/oblib/src/lib/hash/ob_dchash.h @@ -277,23 +277,23 @@ public: } int insert(const key_t& key, Node* node) { +#ifdef ENABLE_DEBUG_LOG common::ObTimeGuard tg("dc_hash::insert", 100 * 1000); +#endif int err = 0; HashNode* pre = NULL; Node key_node(key); { Handle handle(*this, err, 1); - tg.click(); if (0 == (err = handle.search_pre(key_node.hash_, pre))) { - tg.click(); err = _ol_insert((Node*)pre, node); - tg.click(); } } - tg.click(); +#ifdef ENABLE_DEBUG_LOG if (tg.get_diff() > 100000) { _OB_LOG(INFO, "ObDCHash insert cost too much time, click diff (%s)", to_cstring(tg)); } +#endif return err; } diff --git a/deps/oblib/src/rpc/obrpc/ob_rpc_net_handler.cpp b/deps/oblib/src/rpc/obrpc/ob_rpc_net_handler.cpp index 7e75e010f8..e1b02f5502 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_net_handler.cpp +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_net_handler.cpp @@ -23,6 +23,7 @@ #include "lib/allocator/ob_tc_malloc.h" #include "rpc/obrpc/ob_rpc_packet.h" #include "rpc/obrpc/ob_virtual_rpc_protocol_processor.h" +#include "common/ob_clock_generator.h" using namespace oceanbase::common; using namespace oceanbase::common::serialization; @@ -142,7 +143,7 @@ void *ObRpcNetHandler::decode(easy_message_t *ms) ObRpcPacket *pkt = NULL; easy_connection_t *easy_conn = NULL; bool is_current_normal_mode = true; - common::ObTimeGuard timeguard("ObRpcNetHandler::decode", common::OB_EASY_HANDLER_COST_TIME); + const int64_t start_ts = common::ObClockGenerator::getClock(); int ret = OB_SUCCESS; if (OB_ISNULL(ms)) { ret = OB_INVALID_ARGUMENT; @@ -191,15 +192,15 @@ void *ObRpcNetHandler::decode(easy_message_t *ms) LOG_ERROR("failed to decode", K(easy_conn), KP(ms), K(is_current_normal_mode), K(ret)); } else { if (NULL != pkt) { - const int64_t receive_ts = common::ObTimeUtility::current_time(); + const int64_t receive_ts = common::ObClockGenerator::getClock(); const int64_t fly_ts = receive_ts - pkt->get_timestamp(); if (!pkt->is_resp() && fly_ts > common::OB_MAX_PACKET_FLY_TS && TC_REACH_TIME_INTERVAL(100 * 1000)) { LOG_WARN_RET(common::OB_ERR_TOO_MUCH_TIME, "packet fly cost too much time", "pcode", pkt->get_pcode(), "fly_ts", fly_ts, "send_timestamp", pkt->get_timestamp(), "connection", easy_connection_str(ms->c)); } pkt->set_receive_ts(receive_ts); - if (timeguard.get_diff() > common::OB_MAX_PACKET_DECODE_TS && TC_REACH_TIME_INTERVAL(100 * 1000)) { - LOG_WARN_RET(OB_ERR_TOO_MUCH_TIME, "packet decode cost too much time", "pcode", pkt->get_pcode(), K(timeguard), "connection", easy_connection_str(ms->c)); + if (receive_ts - start_ts > common::OB_MAX_PACKET_DECODE_TS && TC_REACH_TIME_INTERVAL(100 * 1000)) { + LOG_WARN_RET(OB_ERR_TOO_MUCH_TIME, "packet decode cost too much time", "pcode", pkt->get_pcode(), "connection", easy_connection_str(ms->c)); } } else { //receive data is not enough @@ -256,7 +257,7 @@ int ObRpcNetHandler::encode(easy_request_t *req, void *packet) req->trace_id[3] = trace_id[3]; } - const int64_t receive_ts = common::ObTimeUtility::current_time(); + const int64_t receive_ts = common::ObClockGenerator::getClock(); const int64_t wait_ts = receive_ts - pkt->get_timestamp(); if (!pkt->is_resp() && wait_ts > common::OB_MAX_PACKET_FLY_TS && TC_REACH_TIME_INTERVAL(100 * 1000)) { diff --git a/src/sql/engine/aggregate/ob_aggregate_function.cpp b/src/sql/engine/aggregate/ob_aggregate_function.cpp index e887e66d21..f446d9487d 100644 --- a/src/sql/engine/aggregate/ob_aggregate_function.cpp +++ b/src/sql/engine/aggregate/ob_aggregate_function.cpp @@ -164,7 +164,7 @@ int ObGroupConcatRowStore::init(const uint64_t tenant_id, iter_idx_ = 0; if (sort_columns.empty()) { int64_t sort_area_size = 0; - if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size(SORT_WORK_AREA, tenant_id, sort_area_size))) { + if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size(SORT_WORK_AREA, tenant_id, NULL, sort_area_size))) { LOG_WARN("failed to get workarea size", K(ret), K(tenant_id)); } else if (OB_FAIL(rs_.init(sort_area_size, tenant_id, ObCtxIds::WORK_AREA, ObModIds::OB_SQL_AGGR_FUN_GROUP_CONCAT, diff --git a/src/sql/engine/aggregate/ob_aggregate_processor.cpp b/src/sql/engine/aggregate/ob_aggregate_processor.cpp index b1d56e5a91..ad9edfed98 100644 --- a/src/sql/engine/aggregate/ob_aggregate_processor.cpp +++ b/src/sql/engine/aggregate/ob_aggregate_processor.cpp @@ -353,7 +353,7 @@ int ObAggregateProcessor::GroupConcatExtraResult::init(const uint64_t tenant_id, } else { int64_t sort_area_size = 0; if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size( - SORT_WORK_AREA, tenant_id, sort_area_size))) { + SORT_WORK_AREA, tenant_id, &eval_ctx.exec_ctx_, sort_area_size))) { LOG_WARN("failed to get workarea size", K(ret), K(tenant_id)); } else if (OB_FAIL(row_store_.init(sort_area_size, tenant_id, diff --git a/src/sql/engine/join/ob_hash_join_op.cpp b/src/sql/engine/join/ob_hash_join_op.cpp index 3c447c4b5c..8f334ec113 100644 --- a/src/sql/engine/join/ob_hash_join_op.cpp +++ b/src/sql/engine/join/ob_hash_join_op.cpp @@ -1417,7 +1417,7 @@ int ObHashJoinOp::get_max_memory_size(int64_t input_size) int64_t extra_memory_size = get_extra_memory_size(); int64_t memory_size = (extra_memory_size + input_size) < 0 ? input_size : (extra_memory_size + input_size); if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size( - ObSqlWorkAreaType::HASH_WORK_AREA, tenant_id, hash_area_size))) { + ObSqlWorkAreaType::HASH_WORK_AREA, tenant_id, &ctx_, hash_area_size))) { LOG_WARN("failed to get workarea size", K(ret), K(tenant_id)); } else if (FALSE_IT(remain_data_memory_size_ = hash_area_size * 80 / 100)) { // default data memory size: 80% @@ -1670,7 +1670,7 @@ int ObHashJoinOp::get_processor_type() int64_t hash_area_size = 0; if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size( ObSqlWorkAreaType::HASH_WORK_AREA, - ctx_.get_my_session()->get_effective_tenant_id(), hash_area_size))) { + ctx_.get_my_session()->get_effective_tenant_id(), &ctx_, hash_area_size))) { LOG_WARN("failed to get workarea size", K(ret)); } remain_data_memory_size_ = hash_area_size * 10; diff --git a/src/sql/engine/ob_sql_mem_mgr_processor.cpp b/src/sql/engine/ob_sql_mem_mgr_processor.cpp index 8512e0b8e8..6492a90269 100644 --- a/src/sql/engine/ob_sql_mem_mgr_processor.cpp +++ b/src/sql/engine/ob_sql_mem_mgr_processor.cpp @@ -92,7 +92,7 @@ int ObSqlMemMgrProcessor::init( int64_t max_mem_size = MAX_SQL_MEM_SIZE; if (OB_FAIL(ret)) { } else if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size( - profile_.get_work_area_type(), tenant_id_, max_mem_size))) { + profile_.get_work_area_type(), tenant_id_, exec_ctx, max_mem_size))) { LOG_WARN("failed to get workarea size", K(ret), K(tenant_id_), K(max_mem_size)); } if (!profile_.get_auto_policy()) { @@ -305,24 +305,47 @@ int ObSqlMemMgrProcessor::alloc_dir_id(int64_t &dir_id) return ret; } -int ObSqlWorkareaUtil::get_workarea_size(const ObSqlWorkAreaType wa_type, const int64_t tenant_id, int64_t &value) +int ObSqlWorkareaUtil::get_workarea_size(const ObSqlWorkAreaType wa_type, + const int64_t tenant_id, + ObExecContext *exec_ctx, + int64_t &value) { int ret = OB_SUCCESS; - ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); - if (tenant_config.is_valid()) { - if (HASH_WORK_AREA == wa_type) { - value = tenant_config->_hash_area_size; - } else if (SORT_WORK_AREA == wa_type) { - value = tenant_config->_sort_area_size; + + if (NULL != exec_ctx) { + if (OB_ISNULL(exec_ctx->get_my_session())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected exec_ctx or session", K(ret), K(wa_type), K(tenant_id), KP(exec_ctx)); + } else { + if (HASH_WORK_AREA == wa_type) { + value = exec_ctx->get_my_session()->get_tenant_hash_area_size(); + } else if (SORT_WORK_AREA == wa_type) { + value = exec_ctx->get_my_session()->get_tenant_sort_area_size(); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: workarea type", K(wa_type), K(tenant_id)); + } + } + } else { + ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); + if (tenant_config.is_valid()) { + if (HASH_WORK_AREA == wa_type) { + value = tenant_config->_hash_area_size; + } else if (SORT_WORK_AREA == wa_type) { + value = tenant_config->_sort_area_size; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: workarea type", K(wa_type), K(tenant_id)); + } + LOG_DEBUG("debug workarea size", K(value), K(tenant_id), K(lbt())); } else { ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected status: workarea type", K(wa_type), K(tenant_id)); + LOG_WARN("failed to init tenant config", K(tenant_id), K(ret)); } - LOG_DEBUG("debug workarea size", K(value), K(tenant_id), K(lbt())); - } else { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("failed to init tenant config", K(tenant_id), K(ret)); } + + LOG_DEBUG("debug workarea size", K(value), K(tenant_id), K(lbt())); + return ret; } diff --git a/src/sql/engine/ob_sql_mem_mgr_processor.h b/src/sql/engine/ob_sql_mem_mgr_processor.h index 9cce5ffc48..be4a1783da 100644 --- a/src/sql/engine/ob_sql_mem_mgr_processor.h +++ b/src/sql/engine/ob_sql_mem_mgr_processor.h @@ -183,6 +183,7 @@ public: static int get_workarea_size( const ObSqlWorkAreaType wa_type, const int64_t tenant_id, + ObExecContext *exec_ctx, int64_t &value ); }; diff --git a/src/sql/ob_sql.cpp b/src/sql/ob_sql.cpp index 2bad40a022..26c30c78a2 100644 --- a/src/sql/ob_sql.cpp +++ b/src/sql/ob_sql.cpp @@ -4009,11 +4009,10 @@ int ObSql::pc_add_plan(ObPlanCacheCtx &pc_ctx, pc_ctx.fp_result_.pc_key_.namespace_ = ObLibCacheNameSpace::NS_CRSR; plan_added = false; bool is_batch_exec = pc_ctx.sql_ctx_.multi_stmt_item_.is_batched_multi_stmt(); - omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); if (OB_ISNULL(phy_plan) || OB_ISNULL(plan_cache)) { ret = OB_NOT_INIT; LOG_WARN("Fail to generate plan", K(phy_plan), K(plan_cache)); - } else if (!tenant_config.is_valid()) { + } else if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant config is invalid", K(ret)); } else if (OB_USE_PLAN_CACHE_NONE == phy_plan->get_phy_plan_hint().plan_cache_policy_) { @@ -4038,7 +4037,7 @@ int ObSql::pc_add_plan(ObPlanCacheCtx &pc_ctx, phy_plan->stat_.db_id_ = pc_ctx.sql_ctx_.spm_ctx_.bl_key_.db_id_; phy_plan->stat_.is_rewrite_sql_ = pc_ctx.is_rewrite_sql_; phy_plan->stat_.rule_version_ = rule_mgr->get_rule_version(); - phy_plan->stat_.enable_udr_ = tenant_config->enable_user_defined_rewrite_rules; + phy_plan->stat_.enable_udr_ = pc_ctx.sql_ctx_.session_info_->enable_user_defined_rewrite_rules(); if (PC_PS_MODE == pc_ctx.mode_ || PC_PL_MODE == pc_ctx.mode_) { //远程SQL第二次进入plan,将raw_sql作为pc_key存入plan cache中, diff --git a/src/sql/plan_cache/ob_plan_cache.cpp b/src/sql/plan_cache/ob_plan_cache.cpp index 28f747bc05..1607b378f7 100644 --- a/src/sql/plan_cache/ob_plan_cache.cpp +++ b/src/sql/plan_cache/ob_plan_cache.cpp @@ -370,22 +370,19 @@ int ObPlanCache::check_after_get_plan(int tmp_ret, ObPhysicalPlan *plan = NULL; bool need_late_compilation = false; ObJITEnableMode jit_mode = ObJITEnableMode::OFF; - omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + bool enable_user_defined_rewrite_rules = false; ObPlanCacheCtx &pc_ctx = static_cast(ctx); if (cache_obj != NULL && ObLibCacheNameSpace::NS_CRSR == cache_obj->get_ns()) { plan = static_cast(cache_obj); } if (OB_SUCC(ret)) { - if (!tenant_config.is_valid()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tenant config is invalid", K(ret)); - } else if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_)) { + if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected null session info", K(ret)); } else if (OB_FAIL(pc_ctx.sql_ctx_.session_info_->get_jit_enabled_mode(jit_mode))) { LOG_WARN("failed to get jit mode"); } else { - // do nothing + enable_user_defined_rewrite_rules = pc_ctx.sql_ctx_.session_info_->enable_user_defined_rewrite_rules(); } } if (OB_SUCC(ret) && plan != NULL) { @@ -394,17 +391,17 @@ int ObPlanCache::check_after_get_plan(int tmp_ret, // when the global rule version changes or enable_user_defined_rewrite_rules changes // it is necessary to check whether the physical plan are expired if ((plan->get_rule_version() != rule_mgr->get_rule_version() - || plan->is_enable_udr() != tenant_config->enable_user_defined_rewrite_rules)) { + || plan->is_enable_udr() != enable_user_defined_rewrite_rules)) { if (OB_FAIL(rule_mgr->fuzzy_check_by_pattern_digest(pc_ctx.get_normalized_pattern_digest(), is_exists))) { LOG_WARN("failed to fuzzy check by pattern digest", K(ret)); } else if (is_exists || plan->is_rewrite_sql()) { ret = OB_OLD_SCHEMA_VERSION; LOG_TRACE("Obsolete user-defined rewrite rules require eviction plan", K(ret), - K(is_exists), K(pc_ctx.raw_sql_), K(plan->is_enable_udr()), K(tenant_config->enable_user_defined_rewrite_rules), + K(is_exists), K(pc_ctx.raw_sql_), K(plan->is_enable_udr()), K(enable_user_defined_rewrite_rules), K(plan->is_rewrite_sql()), K(plan->get_rule_version()), K(rule_mgr->get_rule_version())); } else { plan->set_rule_version(rule_mgr->get_rule_version()); - plan->set_is_enable_udr(tenant_config->enable_user_defined_rewrite_rules); + plan->set_is_enable_udr(enable_user_defined_rewrite_rules); } } if (OB_SUCC(ret)) { diff --git a/src/sql/session/ob_sql_session_info.cpp b/src/sql/session/ob_sql_session_info.cpp index 2d7d95fec5..e5cc7b2fbd 100644 --- a/src/sql/session/ob_sql_session_info.cpp +++ b/src/sql/session/ob_sql_session_info.cpp @@ -2312,6 +2312,7 @@ void ObSQLSessionInfo::ObCachedTenantConfigInfo::refresh() } // 4.sort area size ATOMIC_STORE(&sort_area_size_, tenant_config->_sort_area_size); + ATOMIC_STORE(&hash_area_size_, tenant_config->_hash_area_size); // 5.allow security audit if (OB_SUCCESS != (tmp_ret = ObSecurityAuditUtils::check_allow_audit(*session_, at_type_))) { LOG_WARN_RET(tmp_ret, "fail get tenant_config", "ret", tmp_ret, diff --git a/src/sql/session/ob_sql_session_info.h b/src/sql/session/ob_sql_session_info.h index 1b9b94bc77..20dc70671d 100644 --- a/src/sql/session/ob_sql_session_info.h +++ b/src/sql/session/ob_sql_session_info.h @@ -491,6 +491,8 @@ public: px_join_skew_minfreq_(30), at_type_(ObAuditTrailType::NONE), sort_area_size_(128*1024*1024), + hash_area_size_(128*1024*1024), + enable_user_defined_rewrite_rules_(false), print_sample_ppm_(0), last_check_ec_ts_(0), session_(session) @@ -504,9 +506,11 @@ public: bool get_enable_sql_extension() const { return enable_sql_extension_; } ObAuditTrailType get_at_type() const { return at_type_; } int64_t get_sort_area_size() const { return ATOMIC_LOAD(&sort_area_size_); } + int64_t get_hash_area_size() const { return ATOMIC_LOAD(&hash_area_size_); } int64_t get_print_sample_ppm() const { return ATOMIC_LOAD(&print_sample_ppm_); } bool get_px_join_skew_handling() const { return px_join_skew_handling_; } int64_t get_px_join_skew_minfreq() const { return px_join_skew_minfreq_; } + bool enable_user_defined_rewrite_rules() const { return enable_user_defined_rewrite_rules_; } private: //租户级别配置项缓存session 上,避免每次获取都需要刷新 bool is_external_consistent_; @@ -518,6 +522,8 @@ public: int64_t px_join_skew_minfreq_; ObAuditTrailType at_type_; int64_t sort_area_size_; + int64_t hash_area_size_; + bool enable_user_defined_rewrite_rules_; // for record sys config print_sample_ppm int64_t print_sample_ppm_; int64_t last_check_ec_ts_; @@ -971,11 +977,21 @@ public: at_type = cached_tenant_config_info_.get_at_type(); return common::OB_SUCCESS; } + int64_t get_tenant_hash_area_size() + { + cached_tenant_config_info_.refresh(); + return cached_tenant_config_info_.get_hash_area_size(); + } int64_t get_tenant_sort_area_size() { cached_tenant_config_info_.refresh(); return cached_tenant_config_info_.get_sort_area_size(); } + bool enable_user_defined_rewrite_rules() + { + cached_tenant_config_info_.refresh(); + return cached_tenant_config_info_.enable_user_defined_rewrite_rules(); + } int64_t get_tenant_print_sample_ppm() { cached_tenant_config_info_.refresh(); diff --git a/src/sql/udr/ob_udr_utils.cpp b/src/sql/udr/ob_udr_utils.cpp index fd5c2f9919..fb6cd1ad46 100644 --- a/src/sql/udr/ob_udr_utils.cpp +++ b/src/sql/udr/ob_udr_utils.cpp @@ -241,12 +241,8 @@ int ObUDRUtils::match_udr_and_refill_ctx(const ObString &pattern, int ret = OB_SUCCESS; is_match_udr = false; ObSQLSessionInfo &session = result.get_session(); - omt::ObTenantConfigGuard tenant_config(TENANT_CONF(session.get_effective_tenant_id())); - if (!tenant_config.is_valid()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tenant config is invalid", K(ret)); - } else if (tenant_config->enable_user_defined_rewrite_rules - && !(pc_ctx.is_inner_sql() || PC_PL_MODE == pc_ctx.mode_)) { + if (session.enable_user_defined_rewrite_rules() + && !(pc_ctx.is_inner_sql() || PC_PL_MODE == pc_ctx.mode_)) { ObIAllocator &allocator = result.get_mem_pool(); PatternConstConsList cst_cons_list; if (OB_FAIL(match_udr_item(pattern, session, allocator, item_guard, &cst_cons_list))) { diff --git a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp index 03ab81ab50..0d806a96e4 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp +++ b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.cpp @@ -180,18 +180,30 @@ int ObITransCallback::remove() } ObTransCallbackMgr::WRLockGuard::WRLockGuard(const SpinRWLock &rwlock) +#ifdef ENABLE_DEBUG_LOG : time_guard_(5 * 1000 * 1000), // 5 second lock_guard_(rwlock) { time_guard_.click(); } +#else + : lock_guard_(rwlock) +{ +} +#endif ObTransCallbackMgr::RDLockGuard::RDLockGuard(const SpinRWLock &rwlock) +#ifdef ENABLE_DEBUG_LOG : time_guard_(5 * 1000 * 1000), // 5 second lock_guard_(rwlock) { time_guard_.click(); } +#else + : lock_guard_(rwlock) +{ +} +#endif void ObTransCallbackMgr::reset() { diff --git a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h index 25580c38dd..f9b8f54a88 100644 --- a/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h +++ b/src/storage/memtable/mvcc/ob_mvcc_trans_ctx.h @@ -152,7 +152,9 @@ public: explicit WRLockGuard(const common::SpinRWLock &rwlock); ~WRLockGuard() {} private: +#ifdef ENABLE_DEBUG_LOG common::ObSimpleTimeGuard time_guard_; // print log and lbt, if the lock is held too much time. +#endif common::SpinWLockGuard lock_guard_; }; class RDLockGuard @@ -161,7 +163,9 @@ public: explicit RDLockGuard(const common::SpinRWLock &rwlock); ~RDLockGuard() {} private: +#ifdef ENABLE_DEBUG_LOG common::ObSimpleTimeGuard time_guard_; // print log and lbt, if the lock is held too much time. +#endif common::SpinRLockGuard lock_guard_; }; diff --git a/src/storage/tablet/ob_tablet_create_delete_helper.cpp b/src/storage/tablet/ob_tablet_create_delete_helper.cpp index 9a0c9b81df..8f0f163165 100644 --- a/src/storage/tablet/ob_tablet_create_delete_helper.cpp +++ b/src/storage/tablet/ob_tablet_create_delete_helper.cpp @@ -1414,7 +1414,6 @@ int ObTabletCreateDeleteHelper::check_and_get_tablet( int ret = OB_SUCCESS; ObTabletHandle tablet_handle; ObTabletStatus::Status tablet_status = ObTabletStatus::MAX; - ObTimeGuard time_guard(__func__, 5 * 1000 * 1000); // 5s // TODO(bowen.gbw): optimize this logic, refactor ObTabletStatusChecker if (OB_FAIL(get_tablet(key, tablet_handle, timeout_us))) { @@ -1423,14 +1422,12 @@ int ObTabletCreateDeleteHelper::check_and_get_tablet( } else { LOG_WARN("failed to get tablet", K(ret), K(key), K(timeout_us)); } - } else if (FALSE_IT(time_guard.click("DirectGet"))) { } else if (tablet_handle.get_obj()->is_ls_inner_tablet()) { // no need to check ls inner tablet, do nothing } else if (ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US == timeout_us) { // no checking } else if (OB_FAIL(tablet_handle.get_obj()->get_tablet_status(tablet_status))) { LOG_WARN("failed to get tablet status", K(ret)); - } else if (FALSE_IT(time_guard.click("GetTxData"))) { } else if (ObTabletCommon::DIRECT_GET_COMMITTED_TABLET_TIMEOUT_US == timeout_us) { if (ObTabletStatus::NORMAL != tablet_status) { ret = OB_TABLET_NOT_EXIST; @@ -1445,7 +1442,6 @@ int ObTabletCreateDeleteHelper::check_and_get_tablet( if (OB_FAIL(checker.check(timeout_us))) { LOG_WARN("failed to check tablet status", K(ret), K(timeout_us), K(tablet_handle)); } - time_guard.click("CheckStatus"); } } diff --git a/src/storage/tx/ob_ctx_tx_data.cpp b/src/storage/tx/ob_ctx_tx_data.cpp index 07eddd462e..65c26eb3fe 100644 --- a/src/storage/tx/ob_ctx_tx_data.cpp +++ b/src/storage/tx/ob_ctx_tx_data.cpp @@ -79,31 +79,23 @@ void ObCtxTxData::destroy() int ObCtxTxData::insert_into_tx_table() { int ret = OB_SUCCESS; - common::ObTimeGuard tg("part_ctx::insert_into_tx_table", 100 * 1000); WLockGuard guard(lock_); - tg.click(); if (OB_FAIL(check_tx_data_writable_())) { TRANS_LOG(WARN, "tx data is not writeable", K(ret)); } else { - tg.click(); ObTxTable *tx_table = nullptr; GET_TX_TABLE_(tx_table) if (OB_FAIL(ret)) { } else { - tg.click(); tx_commit_data_ = *(tx_data_guard_.tx_data()); if (OB_FAIL(insert_tx_data_(tx_table, tx_data_guard_.tx_data()))) { TRANS_LOG(WARN, "insert tx data failed", K(ret), K(*this)); } else { - tg.click(); read_only_ = true; } } } - if (tg.get_diff() > 100000) { - TRANS_LOG(INFO, "ObCtxData insert into tx table const too much time", K(tg)); - } return ret; } diff --git a/src/storage/tx/ob_trans_part_ctx.cpp b/src/storage/tx/ob_trans_part_ctx.cpp index 87178ea98a..14470aa951 100644 --- a/src/storage/tx/ob_trans_part_ctx.cpp +++ b/src/storage/tx/ob_trans_part_ctx.cpp @@ -1751,7 +1751,6 @@ int ObPartTransCtx::on_dist_end_(const bool commit) int ObPartTransCtx::on_success(ObTxLogCb *log_cb) { - common::ObTimeGuard tg("part_ctx::on_success", 100 * 1000); int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; const int64_t cur_ts = ObTimeUtility::current_time(); @@ -1759,7 +1758,6 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) ObTransStatistic::get_instance().add_clog_sync_count(tenant_id_, 1); { CtxLockGuard guard(lock_); - tg.click(); const SCN log_ts = log_cb->get_log_ts(); const ObTxLogType last_log_type = log_cb->get_last_log_type(); if (is_exiting_) { @@ -1776,7 +1774,6 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) // save the first error code int save_ret = OB_SUCCESS; ObTxLogCb *cur_cb = busy_cbs_.get_first(); - tg.click(); for (int64_t i = 0; i < busy_cbs_.get_size(); i++) { if (cur_cb->is_callbacked()) { // do nothing @@ -1801,7 +1798,6 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) cur_cb = cur_cb->get_next(); } } - tg.click(); if (cur_cb != log_cb) { ret = OB_ERR_UNEXPECTED; TRANS_LOG(ERROR, "unexpected log callback", K(ret), K(*this), K(*cur_cb), K(*log_cb)); @@ -1812,25 +1808,18 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) } // ignore ret return_log_cb_(log_cb); - tg.click(); if (need_record_log_()) { // ignore error if (OB_SUCCESS != (tmp_ret = submit_record_log_())) { TRANS_LOG(WARN, "failed to submit record log", K(tmp_ret), K(*this)); } } - tg.click(); if (!ObTxLogTypeChecker::is_state_log(last_log_type)) { try_submit_next_log_(); } - tg.click(); // REC_TRANS_TRACE_EXT(tlog_, on_succ_cb, OB_ID(ret), ret, // OB_ID(t), log_ts, // OB_ID(ctx_ref), get_ref()); - if (tg.get_diff() > 100000) { - FORCE_PRINT_TRACE(tlog_, "[tx cb debug] "); - TRANS_LOG(INFO, "on success cost too much time", K_(trans_id), K(last_log_type), K(log_ts), K(tg)); - } } if (OB_SUCCESS != (tmp_ret = ls_tx_ctx_mgr_->revert_tx_ctx_without_lock(this))) { TRANS_LOG(ERROR, "release ctx ref failed", KR(tmp_ret)); @@ -1840,7 +1829,6 @@ int ObPartTransCtx::on_success(ObTxLogCb *log_cb) int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) { - common::ObTimeGuard tg("part_ctx::on_success_ops", 100 * 1000); int ret = OB_SUCCESS; const SCN log_ts = log_cb->get_log_ts(); const palf::LSN log_lsn = log_cb->get_lsn(); @@ -1850,7 +1838,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) if (OB_FAIL(common_on_success_(log_cb))) { TRANS_LOG(WARN, "common_on_success_ failed", K(ret)); } - tg.click(); for (int64_t i = 0; OB_SUCC(ret) && i < cb_arg_array.count(); i++) { const ObTxLogType log_type = cb_arg_array.at(i).get_log_type(); if (ObTxLogType::TX_REDO_LOG == log_type) { @@ -1889,9 +1876,7 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) } TRANS_LOG(INFO, "apply commit info log", KR(ret), K(*this), K(two_phase_log_type)); } - tg.click(); } else if (ObTxLogType::TX_ROLLBACK_TO_LOG == log_type) { - tg.click(); ObTxData *tx_data = log_cb->get_tx_data(); if (OB_ISNULL(tx_data)) { ret = OB_ERR_UNEXPECTED; @@ -1907,7 +1892,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) log_cb->set_tx_data(nullptr); } } - tg.click(); } else if (ObTxLogTypeChecker::is_state_log(log_type)) { sub_state_.clear_state_log_submitting(); if (ObTxLogType::TX_PREPARE_LOG == log_type) { @@ -1933,9 +1917,7 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) } else if (OB_FAIL(ObTxCycleTwoPhaseCommitter::apply_log(two_phase_log_type))) { TRANS_LOG(ERROR, "dist tx apply log failed", KR(ret), K(*this)); } - tg.click(); } else if (ObTxLogType::TX_COMMIT_LOG == log_type) { - tg.click(); if (exec_info_.multi_data_source_.count() > 0 && get_retain_cause() == RetainCause::UNKOWN && OB_FAIL(insert_into_retain_ctx_mgr_(RetainCause::MDS_WAIT_GC_COMMIT_LOG, log_ts, log_lsn, false))) { TRANS_LOG(WARN, "insert into retain_ctx_mgr failed", K(ret), KPC(log_cb), KPC(this)); @@ -1944,20 +1926,17 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) if (OB_FAIL(ctx_tx_data_.set_end_log_ts(log_ts))) { TRANS_LOG(WARN, "set end log ts failed", K(ret)); } else { - tg.click(); if (OB_FAIL(on_local_commit_tx_())) { TRANS_LOG(WARN, "on local commit failed", KR(ret), K(*this)); } } } else { - tg.click(); const NotifyType type = NotifyType::ON_COMMIT; if (OB_FAIL(ctx_tx_data_.set_end_log_ts(log_ts))) { TRANS_LOG(WARN, "set end log ts failed", K(ret)); } else if (OB_FAIL(notify_data_source_(type, log_ts, false, exec_info_.multi_data_source_))) { TRANS_LOG(WARN, "notify data source failed", KR(ret), K(*this)); } - tg.click(); ObTwoPhaseCommitLogType two_phase_log_type; if (OB_FAIL(ret)) { } else if (OB_FAIL(switch_log_type_(log_type, two_phase_log_type))) { @@ -1965,7 +1944,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) } else if (OB_FAIL(ObTxCycleTwoPhaseCommitter::apply_log(two_phase_log_type))) { TRANS_LOG(ERROR, "dist tx apply log failed", KR(ret), K(*this), K(two_phase_log_type)); } - tg.click(); } } else if (ObTxLogType::TX_ABORT_LOG == log_type) { if (exec_info_.multi_data_source_.count() > 0 && get_retain_cause() == RetainCause::UNKOWN @@ -1978,7 +1956,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) } else if (OB_FAIL(on_local_abort_tx_())) { TRANS_LOG(WARN, "on local abort failed", KR(ret), K(*this)); } - tg.click(); } else { const NotifyType type = NotifyType::ON_ABORT; tmp_array.reset(); @@ -1989,7 +1966,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) } else if (OB_FAIL(notify_data_source_(type, log_ts, false, tmp_array))) { TRANS_LOG(WARN, "notify data source failed", KR(ret), K(*this)); } - tg.click(); ObTwoPhaseCommitLogType two_phase_log_type; if (OB_FAIL(ret)) { } else if (OB_FAIL(switch_log_type_(log_type, two_phase_log_type))) { @@ -1997,7 +1973,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) } else if (OB_FAIL(ObTxCycleTwoPhaseCommitter::apply_log(two_phase_log_type))) { TRANS_LOG(ERROR, "dist tx apply log failed", KR(ret), K(*this), K(two_phase_log_type)); } - tg.click(); } } else if (ObTxLogType::TX_CLEAR_LOG == log_type) { ObTwoPhaseCommitLogType two_phase_log_type; @@ -2018,10 +1993,6 @@ int ObPartTransCtx::on_success_ops_(ObTxLogCb *log_cb) OB_ID(offset), log_lsn, OB_ID(ctx_ref), get_ref()); } - if (tg.get_diff() > 100000) { - FORCE_PRINT_TRACE(tlog_, "[tx common cb debug] "); - TRANS_LOG(INFO, "on succcess ops cost too much time", K(tg), K(*this)); - } return ret; } @@ -2234,7 +2205,6 @@ int ObPartTransCtx::wait_gts_elapse_commit_version_(bool &need_wait) { int ret = OB_SUCCESS; need_wait = false; - common::ObTimeGuard tg("part_ctx::wait_gts", 100 * 1000); ObITsMgr *ts_mgr = trans_service_->get_ts_mgr(); @@ -2244,18 +2214,13 @@ int ObPartTransCtx::wait_gts_elapse_commit_version_(bool &need_wait) need_wait))) { TRANS_LOG(WARN, "wait gts elapse failed", KR(ret), K(*this)); } else if (need_wait) { - tg.click(); sub_state_.set_gts_waiting(); if (OB_FAIL(acquire_ctx_ref_())) { TRANS_LOG(WARN, "get trans ctx error", KR(ret), K(*this)); } - tg.click(); TRANS_LOG(INFO, "need wait gts elapse", KR(ret), K(*this)); REC_TRANS_TRACE_EXT2(tlog_, wait_gts_elapse, OB_ID(ctx_ref), get_ref()); } - if (tg.get_diff() > 100000) { - TRANS_LOG(INFO, "wait gts debug cost too much time", K(tg), K(*this)); - } return ret; } @@ -6499,7 +6464,6 @@ int ObPartTransCtx::on_local_commit_tx_() { int ret = OB_SUCCESS; bool need_wait = false; - common::ObTimeGuard tg("part_ctx::on_local_commit", 100 * 1000); if (sub_state_.is_gts_waiting()) { ret = OB_ERR_UNEXPECTED; @@ -6509,7 +6473,6 @@ int ObPartTransCtx::on_local_commit_tx_() TRANS_LOG(WARN, "invalid commit version", K(ret), KPC(this)); } else if (OB_FAIL(wait_gts_elapse_commit_version_(need_wait))) { TRANS_LOG(WARN, "wait gts elapse commit version failed", KR(ret), KPC(this)); - } else if (FALSE_IT(tg.click())) { } else if (OB_FAIL(tx_end_(true /*commit*/))) { TRANS_LOG(WARN, "trans end error", KR(ret), "context", *this); } else if (FALSE_IT(elr_handler_.reset_elr_state())) { @@ -6524,7 +6487,6 @@ int ObPartTransCtx::on_local_commit_tx_() } else if (need_wait) { REC_TRANS_TRACE_EXT2(tlog_, wait_gts_elapse, OB_ID(ctx_ref), get_ref()); - } else if (FALSE_IT(tg.click())) { } if (OB_FAIL(ret) || need_wait) { @@ -6533,10 +6495,6 @@ int ObPartTransCtx::on_local_commit_tx_() TRANS_LOG(WARN, "terminate trx after local commit failed", KR(ret), KPC(this)); } - if (tg.get_diff() > 100000) { - TRANS_LOG(INFO, "on local commit cost too much time", K(ret), K(need_wait), K(tg), K(*this)); - } - return ret; } diff --git a/src/storage/tx_table/ob_tx_data_memtable.cpp b/src/storage/tx_table/ob_tx_data_memtable.cpp index f3c0e2cd77..6f70baf6f9 100644 --- a/src/storage/tx_table/ob_tx_data_memtable.cpp +++ b/src/storage/tx_table/ob_tx_data_memtable.cpp @@ -150,7 +150,6 @@ void ObTxDataMemtable::reset() int ObTxDataMemtable::insert(ObTxData *tx_data) { - common::ObTimeGuard tg("tx_data_memtable::insert", 100 * 1000); int ret = OB_SUCCESS; if (IS_NOT_INIT) { ret = OB_ERR_UNEXPECTED; @@ -172,15 +171,10 @@ int ObTxDataMemtable::insert(ObTxData *tx_data) STORAGE_LOG(ERROR, "insert the tx data into tx_data_map_ fail.", KP(tx_data), KPC(tx_data), KR(ret), KP(tx_data_map_)); } else { - tg.click(); // insert_and_get success max_tx_scn_.inc_update(tx_data->end_scn_); atomic_update_(tx_data); ATOMIC_INC(&inserted_cnt_); - tg.click(); - } - if (tg.get_diff() > 100000) { - STORAGE_LOG(INFO, "tx data memtable insert cost too much time", K(tg)); } return ret; diff --git a/src/storage/tx_table/ob_tx_data_table.cpp b/src/storage/tx_table/ob_tx_data_table.cpp index 9c2b10ecd4..d891426bd5 100644 --- a/src/storage/tx_table/ob_tx_data_table.cpp +++ b/src/storage/tx_table/ob_tx_data_table.cpp @@ -367,7 +367,6 @@ void ObTxDataTable::free_undo_status_list_(ObUndoStatusNode *node_ptr) int ObTxDataTable::insert(ObTxData *&tx_data) { int ret = OB_SUCCESS; - common::ObTimeGuard tg("tx_data_table::insert", 100 * 1000); ObTxDataMemtableWriteGuard write_guard; ObTransID tx_id = tx_data->tx_id_; @@ -381,21 +380,14 @@ int ObTxDataTable::insert(ObTxData *&tx_data) ret = OB_INVALID_ARGUMENT; STORAGE_LOG(ERROR, "trying to insert an invalid tx data into tx data table", KR(ret), KP(tx_data), KPC(tx_data)); - } else if (FALSE_IT(tg.click())) { - // do nothing } else if (OB_FAIL(get_memtable_mgr_()->get_all_memtables_for_write(write_guard))) { STORAGE_LOG(WARN, "get all memtables for write fail.", KR(ret), KPC(get_memtable_mgr_())); - } else if (FALSE_IT(tg.click())) { - // do nothing } else if (OB_FAIL(insert_(tx_data, write_guard))) { STORAGE_LOG(WARN, "insert tx data failed.", KR(ret), KPC(tx_data), KP(this), K(tablet_id_)); } else { // successfully insert // TODO : @gengli do not dec ref and set nullptr after insert } - if (tg.get_diff() > 100000) { - STORAGE_LOG(INFO, "ObTxDataTable insert cost too much time", K(tx_id), K(tg)); - } return ret; } @@ -406,7 +398,6 @@ int ObTxDataTable::insert(ObTxData *&tx_data) int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write_guard) { int ret = OB_SUCCESS; - common::ObTimeGuard tg("tx_data_table::insert_", 100 * 1000); bool inserted = false; ObTxDataMemtable *tx_data_memtable = nullptr; ObTableHandleV2 (&memtable_handles)[MAX_TX_DATA_MEMTABLE_CNT] = write_guard.handles_; @@ -421,11 +412,8 @@ int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "tx data memtable is nullptr.", KR(ret), KP(this), K(tablet_id_), K(memtable_handles[i])); - } else if (FALSE_IT(tg.click())) { - // do nothing } else if (tx_data_memtable->get_start_scn() < tx_data->end_scn_ && tx_data_memtable->get_end_scn() >= tx_data->end_scn_) { - tg.click(); if (OB_FAIL(tx_data_memtable->insert(tx_data))) { STORAGE_LOG(WARN, "insert tx data into tx data memtable failed", @@ -441,7 +429,6 @@ int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write STORAGE_LOG(DEBUG, "skip this tx data memtable", KPC(tx_data), KPC(tx_data_memtable)); } } - tg.click(); // If this tx data can not be inserted into all memtables, check if it should be filtered. // We use the start log ts of the first memtable as the filtering time stamp @@ -451,17 +438,12 @@ int ObTxDataTable::insert_(ObTxData *&tx_data, ObTxDataMemtableWriteGuard &write // Filter this tx data. The part trans ctx need to handle this error code because the memory // of tx data need to be freed. STORAGE_LOG(DEBUG, "This tx data is filtered.", K(clog_checkpoint_scn), KPC(tx_data)); - tg.click(); - } else { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "cannot find the correct tx data memtable to insert", KR(ret), KPC(tx_data), K(clog_checkpoint_scn), K(memtable_handles)); } } - if (tg.get_diff() > 100000) { - STORAGE_LOG(INFO, "ObTxDataTable insert_ cost too much time", K(tg)); - } return ret; }