[CP] fix leak of MALLOC_TIME_MONITOR log when ob_log alloc
This commit is contained in:
		
							
								
								
									
										25
									
								
								deps/oblib/src/common/ob_common_utility.cpp
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										25
									
								
								deps/oblib/src/common/ob_common_utility.cpp
									
									
									
									
										vendored
									
									
								
							@ -163,14 +163,33 @@ int64_t ObFatalErrExtraInfoGuard::to_string(char* buf, const int64_t buf_len) co
 | 
			
		||||
  return pos;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
__thread ObBasicTimeGuard *ObBasicTimeGuard::tl_time_guard = NULL;
 | 
			
		||||
int64_t ObBasicTimeGuard::to_string(char *buf, const int64_t buf_len) const
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  int64_t pos = 0;
 | 
			
		||||
  if (click_count_ > 0) {
 | 
			
		||||
    ret = databuff_printf(buf, buf_len, pos, "time dist: %s=%d", click_str_[0], click_[0]);
 | 
			
		||||
    for (int i = 1; OB_SUCC(ret) && i < click_count_; i++) {
 | 
			
		||||
      ret = databuff_printf(buf, buf_len, pos, ", %s=%d", click_str_[i], click_[i]);
 | 
			
		||||
    const int64_t click_count = click_count_ < MAX_CLICK_COUNT ? click_count_ : MAX_CLICK_COUNT;
 | 
			
		||||
    ClickInfo click_infos[click_count];
 | 
			
		||||
    MEMCPY(click_infos, click_infos_, sizeof(click_infos));
 | 
			
		||||
    std::sort(click_infos, click_infos + click_count, ClickInfo::compare);
 | 
			
		||||
    ret = databuff_printf(buf, buf_len, pos,
 | 
			
		||||
        "owner: %s, click_count: %ld, time dist:[%s=%d",
 | 
			
		||||
        owner_, click_count_, click_infos[0].mod_, click_infos[0].cost_time_);
 | 
			
		||||
    for (int i = 1; OB_SUCC(ret) && i < click_count; ++i) {
 | 
			
		||||
      ret = databuff_printf(buf, buf_len, pos, ", %s=%d",
 | 
			
		||||
          click_infos[i].mod_, click_infos[i].cost_time_);
 | 
			
		||||
    }
 | 
			
		||||
    if (OB_SUCC(ret)) {
 | 
			
		||||
      ret = databuff_printf(buf, buf_len, pos, "], seq:[%d",
 | 
			
		||||
          click_infos[0].seq_);
 | 
			
		||||
    }
 | 
			
		||||
    for (int i = 1; OB_SUCC(ret) && i < click_count; ++i) {
 | 
			
		||||
      ret = databuff_printf(buf, buf_len, pos, ", %d",
 | 
			
		||||
          click_infos[i].seq_);
 | 
			
		||||
    }
 | 
			
		||||
    if (OB_SUCC(ret)) {
 | 
			
		||||
      ret = databuff_printf(buf, buf_len, pos, "]");
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  if (OB_FAIL(ret)) pos = 0;
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										79
									
								
								deps/oblib/src/common/ob_common_utility.h
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										79
									
								
								deps/oblib/src/common/ob_common_utility.h
									
									
									
									
										vendored
									
									
								
							@ -67,19 +67,44 @@ inline int64_t get_cur_ts()
 | 
			
		||||
class ObBasicTimeGuard
 | 
			
		||||
{
 | 
			
		||||
public:
 | 
			
		||||
  explicit ObBasicTimeGuard()
 | 
			
		||||
  struct ClickInfo
 | 
			
		||||
  {
 | 
			
		||||
 | 
			
		||||
    static bool compare(const ClickInfo &left, const ClickInfo &right)
 | 
			
		||||
    {
 | 
			
		||||
      return left.seq_ < right.seq_;
 | 
			
		||||
    }
 | 
			
		||||
    int32_t seq_;
 | 
			
		||||
    int32_t cost_time_;
 | 
			
		||||
    const char* mod_;
 | 
			
		||||
  };
 | 
			
		||||
  explicit ObBasicTimeGuard(const char *owner, const char *start, const char *end)
 | 
			
		||||
    : last_time_guard_(tl_time_guard), owner_(owner), start_(start), end_(end)
 | 
			
		||||
  {
 | 
			
		||||
    if (NULL != tl_time_guard) {
 | 
			
		||||
      tl_time_guard->click(start_);
 | 
			
		||||
    }
 | 
			
		||||
    tl_time_guard = this;
 | 
			
		||||
    start_ts_ = get_cur_ts();
 | 
			
		||||
    last_ts_ = start_ts_;
 | 
			
		||||
    click_count_ = 0;
 | 
			
		||||
    MEMSET(click_infos_, 0, sizeof(click_infos_));
 | 
			
		||||
  }
 | 
			
		||||
  void click(const char *mod = NULL)
 | 
			
		||||
  ~ObBasicTimeGuard()
 | 
			
		||||
  {
 | 
			
		||||
    const int64_t cur_ts = get_cur_ts();
 | 
			
		||||
    if (OB_LIKELY(click_count_ < MAX_CLICK_COUNT)) {
 | 
			
		||||
      click_str_[click_count_] = mod;
 | 
			
		||||
      click_[click_count_++] = (int32_t)(cur_ts - last_ts_);
 | 
			
		||||
      last_ts_ = cur_ts;
 | 
			
		||||
    tl_time_guard = last_time_guard_;
 | 
			
		||||
    if (NULL != tl_time_guard) {
 | 
			
		||||
      tl_time_guard->click(end_);
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  static ObBasicTimeGuard *get_tl_time_guard()
 | 
			
		||||
  {
 | 
			
		||||
    return tl_time_guard;
 | 
			
		||||
  }
 | 
			
		||||
  static void time_guard_click(const char *mod)
 | 
			
		||||
  {
 | 
			
		||||
    if (NULL != tl_time_guard) {
 | 
			
		||||
      tl_time_guard->click(mod);
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  int64_t get_start_ts() const
 | 
			
		||||
@ -91,17 +116,51 @@ public:
 | 
			
		||||
    return get_cur_ts() - start_ts_;
 | 
			
		||||
  }
 | 
			
		||||
  int64_t to_string(char *buf, const int64_t buf_len) const;
 | 
			
		||||
private:
 | 
			
		||||
  void click(const char *mod)
 | 
			
		||||
  {
 | 
			
		||||
    const int64_t cur_ts = get_cur_ts();
 | 
			
		||||
    const int64_t cost_time = cur_ts - last_ts_;
 | 
			
		||||
    last_ts_ = cur_ts;
 | 
			
		||||
    int64_t index = 0;
 | 
			
		||||
    bool record_click = true;
 | 
			
		||||
    if (OB_LIKELY(click_count_ < MAX_CLICK_COUNT)) {
 | 
			
		||||
      index = click_count_;
 | 
			
		||||
    } else {
 | 
			
		||||
      int64_t min_cost_time = cost_time;
 | 
			
		||||
      for (int32_t i = 0; i < MAX_CLICK_COUNT; ++i) {
 | 
			
		||||
        if (click_infos_[i].cost_time_ < min_cost_time) {
 | 
			
		||||
          index = i;
 | 
			
		||||
          min_cost_time = click_infos_[i].cost_time_;
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
      record_click = cost_time > min_cost_time;
 | 
			
		||||
    }
 | 
			
		||||
    if (record_click) {
 | 
			
		||||
      click_infos_[index].seq_ = (int32_t)click_count_++;
 | 
			
		||||
      click_infos_[index].cost_time_ = (int32_t)cost_time;
 | 
			
		||||
      click_infos_[index].mod_ = mod;
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
protected:
 | 
			
		||||
  static const int64_t MAX_CLICK_COUNT = 16;
 | 
			
		||||
  static __thread ObBasicTimeGuard *tl_time_guard;
 | 
			
		||||
private:
 | 
			
		||||
  ObBasicTimeGuard *last_time_guard_;
 | 
			
		||||
  int64_t start_ts_;
 | 
			
		||||
  int64_t last_ts_;
 | 
			
		||||
  int64_t click_count_;
 | 
			
		||||
  int32_t click_[MAX_CLICK_COUNT];
 | 
			
		||||
  const char *click_str_[MAX_CLICK_COUNT];
 | 
			
		||||
  const char *owner_;
 | 
			
		||||
  const char *start_;
 | 
			
		||||
  const char *end_;
 | 
			
		||||
  ClickInfo click_infos_[MAX_CLICK_COUNT];
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
#define BASIC_TIME_GUARD_CLICK(mod) ObBasicTimeGuard::time_guard_click(mod)
 | 
			
		||||
 | 
			
		||||
#define BASIC_TIME_GUARD(name, owner) ObBasicTimeGuard name(owner, owner"#start", owner"#end");
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
} // end of namespace common
 | 
			
		||||
} // end of namespace oceanbase
 | 
			
		||||
 | 
			
		||||
#endif /* _OCEABASE_COMMON_OB_COMMON_UTILITY_H_ */
 | 
			
		||||
 | 
			
		||||
@ -17,24 +17,6 @@ using namespace oceanbase::lib;
 | 
			
		||||
using namespace oceanbase::common;
 | 
			
		||||
 | 
			
		||||
volatile int64_t ObMallocTimeMonitor::WARN_THRESHOLD = 100000;
 | 
			
		||||
__thread ObBasicTimeGuard *ObMallocTimeMonitor::tl_time_guard = NULL;
 | 
			
		||||
 | 
			
		||||
ObMallocTimeMonitor::Guard::Guard(const int64_t size, const ObMemAttr &attr)
 | 
			
		||||
  : size_(size), attr_(attr), last_time_guard_(tl_time_guard), time_guard_()
 | 
			
		||||
{
 | 
			
		||||
  tl_time_guard = &time_guard_;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
ObMallocTimeMonitor::Guard::~Guard()
 | 
			
		||||
{
 | 
			
		||||
  int64_t cost_time = time_guard_.get_diff();
 | 
			
		||||
  ObMallocTimeMonitor::get_instance().inc(cost_time);
 | 
			
		||||
  if (OB_UNLIKELY(cost_time > WARN_THRESHOLD)) {
 | 
			
		||||
    LIB_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "[MALLOC_TIME_MONITOR]", K(cost_time), K_(time_guard), K_(attr), K_(size));
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  tl_time_guard = last_time_guard_;
 | 
			
		||||
}
 | 
			
		||||
void ObMallocTimeMonitor::print()
 | 
			
		||||
{
 | 
			
		||||
  char buf[1024] = {'\0'};
 | 
			
		||||
 | 
			
		||||
@ -21,16 +21,6 @@ namespace lib
 | 
			
		||||
{
 | 
			
		||||
class ObMallocTimeMonitor
 | 
			
		||||
{
 | 
			
		||||
public:
 | 
			
		||||
struct Guard
 | 
			
		||||
{
 | 
			
		||||
  Guard(const int64_t size, const ObMemAttr &attr);
 | 
			
		||||
  ~Guard();
 | 
			
		||||
  const int64_t size_;
 | 
			
		||||
  const ObMemAttr &attr_;
 | 
			
		||||
  ObBasicTimeGuard *last_time_guard_;
 | 
			
		||||
  ObBasicTimeGuard time_guard_;
 | 
			
		||||
};
 | 
			
		||||
public:
 | 
			
		||||
  static volatile int64_t WARN_THRESHOLD;
 | 
			
		||||
  static constexpr const int64_t TIME_SLOT[] = {10, 100, 1000, 10000, 100000, 1000000, INT64_MAX};
 | 
			
		||||
@ -45,12 +35,6 @@ public:
 | 
			
		||||
    static ObMallocTimeMonitor instance;
 | 
			
		||||
    return instance;
 | 
			
		||||
  }
 | 
			
		||||
  static void click(const char *mod = NULL)
 | 
			
		||||
  {
 | 
			
		||||
    if (NULL != tl_time_guard) {
 | 
			
		||||
      tl_time_guard->click(mod);
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  void inc(int64_t cost_time)
 | 
			
		||||
  {
 | 
			
		||||
    for (int i = 0; i < TIME_SLOT_NUM; ++i) {
 | 
			
		||||
@ -61,9 +45,21 @@ public:
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  void record_malloc_time(ObBasicTimeGuard& time_guard, const int64_t size, const ObMemAttr& attr)
 | 
			
		||||
  {
 | 
			
		||||
    const int64_t cost_time = time_guard.get_diff();
 | 
			
		||||
    inc(cost_time);
 | 
			
		||||
    if (OB_UNLIKELY(cost_time > WARN_THRESHOLD)) {
 | 
			
		||||
      const int64_t buf_len = 1024;
 | 
			
		||||
      char buf[buf_len] = {'\0'};
 | 
			
		||||
      int64_t pos = attr.to_string(buf, buf_len);
 | 
			
		||||
      (void)logdata_printf(buf, buf_len, pos, ", size=%ld, ", size);
 | 
			
		||||
      pos += time_guard.to_string(buf + pos, buf_len - pos);
 | 
			
		||||
      int64_t tid = GETTID();
 | 
			
		||||
      fprintf(stderr, "[%ld]OB_MALLOC COST TOO MUCH TIME, cost_time=%ld, %.*s\n", tid, cost_time, static_cast<int>(pos), buf);
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  void print();
 | 
			
		||||
private:
 | 
			
		||||
  static __thread ObBasicTimeGuard *tl_time_guard;
 | 
			
		||||
private:
 | 
			
		||||
  int64_t last_total_cost_times_[TIME_SLOT_NUM];
 | 
			
		||||
  int64_t last_counts_[TIME_SLOT_NUM];
 | 
			
		||||
 | 
			
		||||
@ -33,9 +33,9 @@ void *ObTenantCtxAllocator::alloc(const int64_t size, const ObMemAttr &attr)
 | 
			
		||||
  abort_unless(attr.ctx_id_ == ctx_id_);
 | 
			
		||||
  void *ptr = NULL;
 | 
			
		||||
  if (OB_LIKELY(ObSubCtxIds::MAX_SUB_CTX_ID == attr.sub_ctx_id_)) {
 | 
			
		||||
    ptr = common_alloc(size, attr, *this, obj_mgr_);
 | 
			
		||||
    ptr = common_realloc(NULL, size, attr, *this, obj_mgr_);
 | 
			
		||||
  } else if (OB_UNLIKELY(attr.sub_ctx_id_ < ObSubCtxIds::MAX_SUB_CTX_ID)) {
 | 
			
		||||
    ptr = common_alloc(size, attr, *this, obj_mgrs_[attr.sub_ctx_id_]);
 | 
			
		||||
    ptr = common_realloc(NULL, size, attr, *this, obj_mgrs_[attr.sub_ctx_id_]);
 | 
			
		||||
  } else {
 | 
			
		||||
    LIB_LOG_RET(WARN, OB_ERR_UNEXPECTED, "allocate memory with unexpected sub_ctx_id");
 | 
			
		||||
  }
 | 
			
		||||
@ -401,77 +401,6 @@ void ObTenantCtxAllocator::update_wash_stat(int64_t related_chunks, int64_t bloc
 | 
			
		||||
  (void)ATOMIC_FAA(&washed_size_, size);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
template <typename T>
 | 
			
		||||
void* ObTenantCtxAllocator::common_alloc(const int64_t size, const ObMemAttr &attr,
 | 
			
		||||
                                         ObTenantCtxAllocator& ta, T &allocator)
 | 
			
		||||
{
 | 
			
		||||
  SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
 | 
			
		||||
  void *ret = nullptr;
 | 
			
		||||
  AObject *obj = nullptr;
 | 
			
		||||
  int64_t alloc_size = 0;
 | 
			
		||||
  bool sample_allowed = false;
 | 
			
		||||
  bool is_errsim = false;
 | 
			
		||||
  if (!attr.label_.is_valid()) {
 | 
			
		||||
    LIB_LOG_RET(ERROR, OB_INVALID_ARGUMENT, "OB_MOD_DO_NOT_USE_ME ALLOC", K(size));
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
#ifdef ERRSIM
 | 
			
		||||
  const ObErrsimModuleType type = THIS_WORKER.get_module_type();
 | 
			
		||||
  if (is_errsim_module(ta.get_tenant_id(), type.type_)) {
 | 
			
		||||
    //errsim alloc memory failed.
 | 
			
		||||
    obj = nullptr;
 | 
			
		||||
    is_errsim = true;
 | 
			
		||||
  }
 | 
			
		||||
#endif
 | 
			
		||||
 | 
			
		||||
  if (OB_UNLIKELY(is_errsim)) {
 | 
			
		||||
  } else {
 | 
			
		||||
    ObMallocTimeMonitor::Guard guard(size, attr);
 | 
			
		||||
    sample_allowed = ObMallocSampleLimiter::malloc_sample_allowed(size, attr);
 | 
			
		||||
    alloc_size = sample_allowed ? (size + AOBJECT_BACKTRACE_SIZE) : size;
 | 
			
		||||
    obj = allocator.alloc_object(alloc_size, attr);
 | 
			
		||||
    if (OB_ISNULL(obj)) {
 | 
			
		||||
      int64_t total_size = 0;
 | 
			
		||||
      if (g_alloc_failed_ctx().need_wash_block()) {
 | 
			
		||||
        total_size += ta.sync_wash();
 | 
			
		||||
        ObMallocTimeMonitor::click("WASH_BLOCK_END");
 | 
			
		||||
      } else if (g_alloc_failed_ctx().need_wash_chunk()) {
 | 
			
		||||
        total_size += CHUNK_MGR.sync_wash();
 | 
			
		||||
        ObMallocTimeMonitor::click("WASH_CHUNK_END");
 | 
			
		||||
      }
 | 
			
		||||
      if (total_size > 0) {
 | 
			
		||||
        obj = allocator.alloc_object(alloc_size, attr);
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  if (NULL != obj) {
 | 
			
		||||
    obj->on_malloc_sample_ = sample_allowed;
 | 
			
		||||
    ob_malloc_sample_backtrace(obj, size);
 | 
			
		||||
    ret = obj->data_;
 | 
			
		||||
    get_mem_leak_checker().on_alloc(*obj, attr);
 | 
			
		||||
    SANITY_POISON(obj, AOBJECT_HEADER_SIZE);
 | 
			
		||||
    SANITY_UNPOISON(obj->data_, size);
 | 
			
		||||
    SANITY_POISON((void*)upper_align((int64_t)obj->data_ + size, 8),
 | 
			
		||||
                                     alloc_size - size + sizeof(AOBJECT_TAIL_MAGIC_CODE));
 | 
			
		||||
  }
 | 
			
		||||
  if (OB_UNLIKELY(nullptr == obj) && TC_REACH_TIME_INTERVAL(1 * 1000 * 1000)) {
 | 
			
		||||
    int level = ObFreeLogPrinter::get_level();
 | 
			
		||||
    ObFreeLogPrinter::get_instance().enable_free_log(attr.tenant_id_,
 | 
			
		||||
                                                     attr.ctx_id_, level);
 | 
			
		||||
    const char *msg = is_errsim ? "[ERRSIM] errsim inject memory error" : alloc_failed_msg();
 | 
			
		||||
    LOG_DBA_WARN(OB_ALLOCATE_MEMORY_FAILED, "[OOPS]", "alloc failed reason", KCSTRING(msg));
 | 
			
		||||
    _OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "oops, alloc failed, tenant_id=%ld, ctx_id=%ld, ctx_name=%s, ctx_hold=%ld, "
 | 
			
		||||
                "ctx_limit=%ld, tenant_hold=%ld, tenant_limit=%ld",
 | 
			
		||||
                attr.tenant_id_, attr.ctx_id_,
 | 
			
		||||
                get_global_ctx_info().get_ctx_name(attr.ctx_id_),
 | 
			
		||||
                ta.get_hold(), ta.get_limit(), ta.get_tenant_hold(), ta.get_tenant_limit());
 | 
			
		||||
    // 49 is the user defined signal to dump memory
 | 
			
		||||
    raise(49);
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
template <typename T>
 | 
			
		||||
void* ObTenantCtxAllocator::common_realloc(const void *ptr, const int64_t size,
 | 
			
		||||
                                           const ObMemAttr &attr, ObTenantCtxAllocator& ta,
 | 
			
		||||
@ -508,7 +437,8 @@ void* ObTenantCtxAllocator::common_realloc(const void *ptr, const int64_t size,
 | 
			
		||||
 | 
			
		||||
  if (OB_UNLIKELY(is_errsim)) {
 | 
			
		||||
  } else {
 | 
			
		||||
    ObMallocTimeMonitor::Guard guard(size, attr);
 | 
			
		||||
    BASIC_TIME_GUARD(time_guard, "ObMalloc");
 | 
			
		||||
    DEFER(ObMallocTimeMonitor::get_instance().record_malloc_time(time_guard, size, attr));
 | 
			
		||||
    sample_allowed = ObMallocSampleLimiter::malloc_sample_allowed(size, attr);
 | 
			
		||||
    alloc_size = sample_allowed ? (size + AOBJECT_BACKTRACE_SIZE) : size;
 | 
			
		||||
    obj = allocator.realloc_object(obj, alloc_size, attr);
 | 
			
		||||
@ -516,10 +446,10 @@ void* ObTenantCtxAllocator::common_realloc(const void *ptr, const int64_t size,
 | 
			
		||||
      int64_t total_size = 0;
 | 
			
		||||
      if (g_alloc_failed_ctx().need_wash_block()) {
 | 
			
		||||
        total_size += ta.sync_wash();
 | 
			
		||||
        ObMallocTimeMonitor::click("WASH_BLOCK_END");
 | 
			
		||||
        BASIC_TIME_GUARD_CLICK("WASH_BLOCK_END");
 | 
			
		||||
      } else if (g_alloc_failed_ctx().need_wash_chunk()) {
 | 
			
		||||
        total_size += CHUNK_MGR.sync_wash();
 | 
			
		||||
        ObMallocTimeMonitor::click("WASH_CHUNK_END");
 | 
			
		||||
        BASIC_TIME_GUARD_CLICK("WASH_CHUNK_END");
 | 
			
		||||
      }
 | 
			
		||||
      if (total_size > 0) {
 | 
			
		||||
        obj = allocator.realloc_object(obj, alloc_size, attr);
 | 
			
		||||
 | 
			
		||||
@ -299,10 +299,6 @@ private:
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
public:
 | 
			
		||||
  template <typename T>
 | 
			
		||||
  static void* common_alloc(const int64_t size, const ObMemAttr &attr,
 | 
			
		||||
                            ObTenantCtxAllocator& ta, T &allocator);
 | 
			
		||||
 | 
			
		||||
  template <typename T>
 | 
			
		||||
  static void* common_realloc(const void *ptr, const int64_t size,
 | 
			
		||||
                              const ObMemAttr &attr, ObTenantCtxAllocator& ta,
 | 
			
		||||
 | 
			
		||||
@ -37,13 +37,13 @@ void *ObAllocator::alloc(const int64_t size, const ObMemAttr &attr)
 | 
			
		||||
    auto ta = lib::ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(inner_attr.tenant_id_,
 | 
			
		||||
                                                                                inner_attr.ctx_id_);
 | 
			
		||||
    if (OB_LIKELY(NULL != ta)) {
 | 
			
		||||
      ptr = ObTenantCtxAllocator::common_alloc(size, inner_attr, *(ta.ref_allocator()), os_);
 | 
			
		||||
      ptr = ObTenantCtxAllocator::common_realloc(NULL, size, inner_attr, *(ta.ref_allocator()), os_);
 | 
			
		||||
    } else if (FORCE_MALLOC_FOR_ABSENT_TENANT()) {
 | 
			
		||||
      inner_attr.tenant_id_ = OB_SERVER_TENANT_ID;
 | 
			
		||||
      ta = lib::ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(inner_attr.tenant_id_,
 | 
			
		||||
                                                                            inner_attr.ctx_id_);
 | 
			
		||||
      if (NULL != ta) {
 | 
			
		||||
        ptr = ObTenantCtxAllocator::common_alloc(size, inner_attr, *(ta.ref_allocator()), nos_);
 | 
			
		||||
        ptr = ObTenantCtxAllocator::common_realloc(NULL, size, inner_attr, *(ta.ref_allocator()), nos_);
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										12
									
								
								deps/oblib/src/lib/oblog/ob_log.h
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										12
									
								
								deps/oblib/src/lib/oblog/ob_log.h
									
									
									
									
										vendored
									
									
								
							@ -1150,7 +1150,7 @@ inline void ObLogger::do_log_message(const bool is_async,
 | 
			
		||||
  auto fd_type = get_fd_type(mod_name);
 | 
			
		||||
  const int64_t log_size = limited_left_log_size_ + NORMAL_LOG_SIZE;
 | 
			
		||||
  limited_left_log_size_ = 0;
 | 
			
		||||
  ObBasicTimeGuard tg;
 | 
			
		||||
  BASIC_TIME_GUARD(tg, "ObLog");
 | 
			
		||||
  if (FD_TRACE_FILE != fd_type && OB_FAIL(check_tl_log_limiter(location_hash_val, level, errcode, log_size,
 | 
			
		||||
          allow, limiter_info))) {
 | 
			
		||||
    LOG_STDERR("precheck_tl_log_limiter error, ret=%d\n", ret);
 | 
			
		||||
@ -1203,7 +1203,7 @@ inline void ObLogger::do_log_message(const bool is_async,
 | 
			
		||||
        check_log_end(*log_item, pos);
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    tg.click("FORMAT_END");
 | 
			
		||||
    BASIC_TIME_GUARD_CLICK("FORMAT_END");
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
    if (OB_SUCC(ret)) {
 | 
			
		||||
@ -1223,7 +1223,7 @@ _Pragma("GCC diagnostic pop")
 | 
			
		||||
            // update buf_size
 | 
			
		||||
          new_log_item->set_buf_size(log_item->get_data_len());
 | 
			
		||||
          log_item = new_log_item;
 | 
			
		||||
          tg.click("ALLOC_END");
 | 
			
		||||
          BASIC_TIME_GUARD_CLICK("ALLOC_END");
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
        if (OB_SUCC(ret)) {
 | 
			
		||||
@ -1236,12 +1236,12 @@ _Pragma("GCC diagnostic pop")
 | 
			
		||||
              (void)ATOMIC_AAF(current_written_count_ + tl_type, 1);
 | 
			
		||||
            }
 | 
			
		||||
            last_logging_seq_ = curr_logging_seq_;
 | 
			
		||||
            tg.click("APPEND_END");
 | 
			
		||||
            BASIC_TIME_GUARD_CLICK("APPEND_END");
 | 
			
		||||
          }
 | 
			
		||||
        }
 | 
			
		||||
      } else {
 | 
			
		||||
        flush_logs_to_file(&log_item, 1);
 | 
			
		||||
        tg.click("FLUSH_END");
 | 
			
		||||
        BASIC_TIME_GUARD_CLICK("FLUSH_END");
 | 
			
		||||
      }
 | 
			
		||||
 | 
			
		||||
      // stat
 | 
			
		||||
@ -1251,7 +1251,7 @@ _Pragma("GCC diagnostic pop")
 | 
			
		||||
          free_log_item(log_item);
 | 
			
		||||
        }
 | 
			
		||||
        log_item = NULL;
 | 
			
		||||
        tg.click("FREE_END");
 | 
			
		||||
        BASIC_TIME_GUARD_CLICK("FREE_END");
 | 
			
		||||
      }
 | 
			
		||||
      check_reset_force_allows();
 | 
			
		||||
    } /* not allow */
 | 
			
		||||
 | 
			
		||||
@ -75,7 +75,7 @@ AChunk *ObTenantMemoryMgr::alloc_chunk(const int64_t size, const ObMemAttr &attr
 | 
			
		||||
        update_cache_hold(hold_size);
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    ObMallocTimeMonitor::click("ALLOC_CHUNK_END");
 | 
			
		||||
    BASIC_TIME_GUARD_CLICK("ALLOC_CHUNK_END");
 | 
			
		||||
    if (!reach_ctx_limit && NULL != cache_washer_ && NULL == chunk && hold_size < cache_hold_
 | 
			
		||||
        && attr.label_ != ObNewModIds::OB_KVSTORE_CACHE_MB) {
 | 
			
		||||
      // try wash memory from cache
 | 
			
		||||
@ -151,7 +151,7 @@ AChunk *ObTenantMemoryMgr::alloc_chunk(const int64_t size, const ObMemAttr &attr
 | 
			
		||||
          }
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
      ObMallocTimeMonitor::click("WASH_KVCACHE_END");
 | 
			
		||||
      BASIC_TIME_GUARD_CLICK("WASH_KVCACHE_END");
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return chunk;
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										1
									
								
								deps/oblib/unittest/common/CMakeLists.txt
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										1
									
								
								deps/oblib/unittest/common/CMakeLists.txt
									
									
									
									
										vendored
									
									
								
							@ -1,6 +1,7 @@
 | 
			
		||||
#oblib_addtest(test_memfrag_recycle_allocator.cpp)
 | 
			
		||||
oblib_addtest(test_cell_reader.cpp)
 | 
			
		||||
oblib_addtest(test_cell_writer.cpp)
 | 
			
		||||
oblib_addtest(test_common_utility.cpp)
 | 
			
		||||
oblib_addtest(test_delay_free_allocator.cpp)
 | 
			
		||||
#oblib_addtest(test_ext_store_range.cpp)
 | 
			
		||||
#oblib_addtest(test_multi_version_range.cpp)
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										50
									
								
								deps/oblib/unittest/common/test_common_utility.cpp
									
									
									
									
										vendored
									
									
										Normal file
									
								
							
							
						
						
									
										50
									
								
								deps/oblib/unittest/common/test_common_utility.cpp
									
									
									
									
										vendored
									
									
										Normal file
									
								
							@ -0,0 +1,50 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright (c) 2021 OceanBase
 | 
			
		||||
 * OceanBase CE is licensed under Mulan PubL v2.
 | 
			
		||||
 * You can use this software according to the terms and conditions of the Mulan PubL v2.
 | 
			
		||||
 * You may obtain a copy of Mulan PubL v2 at:
 | 
			
		||||
 *          http://license.coscl.org.cn/MulanPubL-2.0
 | 
			
		||||
 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
 | 
			
		||||
 * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
 | 
			
		||||
 * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
 | 
			
		||||
 * See the Mulan PubL v2 for more details.
 | 
			
		||||
 */
 | 
			
		||||
 | 
			
		||||
#include <gtest/gtest.h>
 | 
			
		||||
#define private public
 | 
			
		||||
#include "common/ob_common_utility.h"
 | 
			
		||||
using namespace  oceanbase::common;
 | 
			
		||||
TEST(TestBasicTimeGuard, tl_time_guard)
 | 
			
		||||
{
 | 
			
		||||
  EXPECT_TRUE(NULL == ObBasicTimeGuard::get_tl_time_guard());
 | 
			
		||||
  BASIC_TIME_GUARD(tg, "ObLog");
 | 
			
		||||
  EXPECT_TRUE(&tg == ObBasicTimeGuard::get_tl_time_guard());
 | 
			
		||||
  {
 | 
			
		||||
    BASIC_TIME_GUARD(tg1, "ObMalloc");
 | 
			
		||||
    EXPECT_TRUE(&tg1 == ObBasicTimeGuard::get_tl_time_guard());
 | 
			
		||||
  }
 | 
			
		||||
  EXPECT_TRUE(&tg == ObBasicTimeGuard::get_tl_time_guard());
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
TEST(TestBasicTimeGuard, click_infos)
 | 
			
		||||
{
 | 
			
		||||
  BASIC_TIME_GUARD(tg, "ObMalloc");
 | 
			
		||||
  int index = 8;
 | 
			
		||||
  for (int i = 0; i < 16; ++i) {
 | 
			
		||||
    if (index == i) {
 | 
			
		||||
      usleep(1);
 | 
			
		||||
    } else {
 | 
			
		||||
      usleep(100);
 | 
			
		||||
    }
 | 
			
		||||
    BASIC_TIME_GUARD_CLICK("alloc_chunk");
 | 
			
		||||
  }
 | 
			
		||||
  EXPECT_EQ(index, tg.click_infos_[index].seq_);
 | 
			
		||||
  usleep(50);
 | 
			
		||||
  BASIC_TIME_GUARD_CLICK("target");
 | 
			
		||||
  EXPECT_EQ(16, tg.click_infos_[index].seq_);
 | 
			
		||||
}
 | 
			
		||||
int main(int argc, char *argv[])
 | 
			
		||||
{
 | 
			
		||||
  ::testing::InitGoogleTest(&argc, argv);
 | 
			
		||||
  return RUN_ALL_TESTS();
 | 
			
		||||
}
 | 
			
		||||
		Reference in New Issue
	
	Block a user