[branch-2.1](memory) Allocator support address sanitizers (#40836)
pick #33396 #33862 #33853 #33732 #33841 #33933 #34901 #35014 --------- Co-authored-by: yiguolei <676222867@qq.com> Co-authored-by: yiguolei <yiguolei@gmail.com>
This commit is contained in:
@ -1106,6 +1106,7 @@ DEFINE_mString(kerberos_krb5_conf_path, "/etc/krb5.conf");
|
||||
|
||||
DEFINE_mString(get_stack_trace_tool, "libunwind");
|
||||
DEFINE_mString(dwarf_location_info_mode, "FAST");
|
||||
DEFINE_mBool(enable_address_sanitizers_with_stack_trace, "true");
|
||||
|
||||
// the ratio of _prefetch_size/_batch_size in AutoIncIDBuffer
|
||||
DEFINE_mInt64(auto_inc_prefetch_size_ratio, "10");
|
||||
|
||||
@ -1158,6 +1158,7 @@ DECLARE_mString(kerberos_krb5_conf_path);
|
||||
|
||||
// Values include `none`, `glog`, `boost`, `glibc`, `libunwind`
|
||||
DECLARE_mString(get_stack_trace_tool);
|
||||
DECLARE_mBool(enable_address_sanitizers_with_stack_trace);
|
||||
|
||||
// DISABLED: Don't resolve location info.
|
||||
// FAST: Perform CU lookup using .debug_aranges (might be incomplete).
|
||||
|
||||
@ -39,7 +39,7 @@ EngineAlterTabletTask::EngineAlterTabletTask(const TAlterTabletReqV2& request)
|
||||
->get_storage_engine()
|
||||
->memory_limitation_bytes_per_thread_for_schema_change();
|
||||
_mem_tracker = MemTrackerLimiter::create_shared(
|
||||
MemTrackerLimiter::Type::SCHEMA_CHANGE,
|
||||
MemTrackerLimiter::Type::OTHER,
|
||||
fmt::format("EngineAlterTabletTask#baseTabletId={}:newTabletId={}",
|
||||
std::to_string(_alter_tablet_req.base_tablet_id),
|
||||
std::to_string(_alter_tablet_req.new_tablet_id)),
|
||||
|
||||
@ -81,7 +81,7 @@ EnginePublishVersionTask::EnginePublishVersionTask(
|
||||
_succ_tablets(succ_tablets),
|
||||
_discontinuous_version_tablets(discontinuous_version_tablets),
|
||||
_table_id_to_num_delta_rows(table_id_to_num_delta_rows) {
|
||||
_mem_tracker = MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::SCHEMA_CHANGE,
|
||||
_mem_tracker = MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::OTHER,
|
||||
"TabletPublishTxnTask");
|
||||
}
|
||||
|
||||
@ -370,7 +370,7 @@ TabletPublishTxnTask::TabletPublishTxnTask(EnginePublishVersionTask* engine_task
|
||||
_transaction_id(transaction_id),
|
||||
_version(version),
|
||||
_tablet_info(tablet_info),
|
||||
_mem_tracker(MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::SCHEMA_CHANGE,
|
||||
_mem_tracker(MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::OTHER,
|
||||
"TabletPublishTxnTask")) {
|
||||
_stats.submit_time_us = MonotonicMicros();
|
||||
}
|
||||
|
||||
@ -119,7 +119,7 @@ public:
|
||||
_partition_id(partition_id),
|
||||
_transaction_id(transaction_id),
|
||||
_version(version),
|
||||
_mem_tracker(MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::SCHEMA_CHANGE,
|
||||
_mem_tracker(MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::OTHER,
|
||||
"AsyncTabletPublishTask")) {
|
||||
_stats.submit_time_us = MonotonicMicros();
|
||||
}
|
||||
|
||||
@ -185,6 +185,12 @@ public:
|
||||
std::shared_ptr<MemTrackerLimiter> segcompaction_mem_tracker() {
|
||||
return _segcompaction_mem_tracker;
|
||||
}
|
||||
std::shared_ptr<MemTrackerLimiter> point_query_executor_mem_tracker() {
|
||||
return _point_query_executor_mem_tracker;
|
||||
}
|
||||
std::shared_ptr<MemTrackerLimiter> block_compression_mem_tracker() {
|
||||
return _block_compression_mem_tracker;
|
||||
}
|
||||
std::shared_ptr<MemTrackerLimiter> rowid_storage_reader_tracker() {
|
||||
return _rowid_storage_reader_tracker;
|
||||
}
|
||||
@ -363,6 +369,10 @@ private:
|
||||
// Count the memory consumption of segment compaction tasks.
|
||||
std::shared_ptr<MemTrackerLimiter> _segcompaction_mem_tracker;
|
||||
|
||||
// Tracking memory may be shared between multiple queries.
|
||||
std::shared_ptr<MemTrackerLimiter> _point_query_executor_mem_tracker;
|
||||
std::shared_ptr<MemTrackerLimiter> _block_compression_mem_tracker;
|
||||
|
||||
// TODO, looking forward to more accurate tracking.
|
||||
std::shared_ptr<MemTrackerLimiter> _rowid_storage_reader_tracker;
|
||||
std::shared_ptr<MemTrackerLimiter> _subcolumns_tree_tracker;
|
||||
|
||||
@ -529,6 +529,10 @@ void ExecEnv::init_mem_tracker() {
|
||||
std::make_shared<MemTracker>("IOBufBlockMemory", _details_mem_tracker_set.get());
|
||||
_segcompaction_mem_tracker =
|
||||
MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "SegCompaction");
|
||||
_point_query_executor_mem_tracker =
|
||||
MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "PointQueryExecutor");
|
||||
_block_compression_mem_tracker =
|
||||
MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "BlockCompression");
|
||||
_rowid_storage_reader_tracker =
|
||||
MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "RowIdStorageReader");
|
||||
_subcolumns_tree_tracker =
|
||||
|
||||
@ -160,7 +160,7 @@ Status FoldConstantExecutor::_init(const TQueryGlobals& query_globals,
|
||||
fragment_params.params = params;
|
||||
fragment_params.protocol_version = PaloInternalServiceVersion::V1;
|
||||
_mem_tracker = MemTrackerLimiter::create_shared(
|
||||
MemTrackerLimiter::Type::SCHEMA_CHANGE,
|
||||
MemTrackerLimiter::Type::OTHER,
|
||||
fmt::format("FoldConstant:query_id={}", print_id(_query_id)));
|
||||
_runtime_state =
|
||||
RuntimeState::create_unique(fragment_params.params, query_options, query_globals,
|
||||
|
||||
@ -289,6 +289,7 @@ std::string FragmentMgr::to_http_path(const std::string& file_name) {
|
||||
Status FragmentMgr::trigger_pipeline_context_report(
|
||||
const ReportStatusRequest req, std::shared_ptr<pipeline::PipelineFragmentContext>&& ctx) {
|
||||
return _async_report_thread_pool->submit_func([this, req, ctx]() {
|
||||
SCOPED_ATTACH_TASK(ctx->get_query_ctx()->query_mem_tracker);
|
||||
coordinator_callback(req);
|
||||
if (!req.done) {
|
||||
ctx->refresh_next_report_time();
|
||||
|
||||
@ -39,6 +39,7 @@
|
||||
#include "util/perf_counters.h"
|
||||
#include "util/pretty_printer.h"
|
||||
#include "util/runtime_profile.h"
|
||||
#include "util/stack_util.h"
|
||||
|
||||
namespace doris {
|
||||
|
||||
@ -110,7 +111,7 @@ std::shared_ptr<MemTrackerLimiter> MemTrackerLimiter::create_shared(MemTrackerLi
|
||||
MemTrackerLimiter::~MemTrackerLimiter() {
|
||||
consume(_untracked_mem);
|
||||
static std::string mem_tracker_inaccurate_msg =
|
||||
", mem tracker not equal to 0 when mem tracker destruct, this usually means that "
|
||||
"mem tracker not equal to 0 when mem tracker destruct, this usually means that "
|
||||
"memory tracking is inaccurate and SCOPED_ATTACH_TASK and "
|
||||
"SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER are not used correctly. "
|
||||
"1. For query and load, memory leaks may have occurred, it is expected that the query "
|
||||
@ -124,21 +125,90 @@ MemTrackerLimiter::~MemTrackerLimiter() {
|
||||
"4. If you need to "
|
||||
"transfer memory tracking value between two trackers, can use transfer_to.";
|
||||
if (_consumption->current_value() != 0) {
|
||||
// TODO, expect mem tracker equal to 0 at the task end.
|
||||
if (doris::config::enable_memory_orphan_check && _type == Type::QUERY) {
|
||||
LOG(INFO) << "mem tracker label: " << _label
|
||||
<< ", consumption: " << _consumption->current_value()
|
||||
<< ", peak consumption: " << _consumption->peak_value()
|
||||
<< mem_tracker_inaccurate_msg;
|
||||
// TODO, expect mem tracker equal to 0 at the load/compaction/etc. task end.
|
||||
#ifndef NDEBUG
|
||||
if (_type == Type::QUERY) {
|
||||
std::string err_msg =
|
||||
fmt::format("mem tracker label: {}, consumption: {}, peak consumption: {}, {}.",
|
||||
label(), _consumption->current_value(), _consumption->peak_value(),
|
||||
mem_tracker_inaccurate_msg);
|
||||
LOG(FATAL) << err_msg << print_address_sanitizers();
|
||||
}
|
||||
#endif
|
||||
if (ExecEnv::tracking_memory()) {
|
||||
ExecEnv::GetInstance()->orphan_mem_tracker()->consume(_consumption->current_value());
|
||||
}
|
||||
_consumption->set(0);
|
||||
#ifndef NDEBUG
|
||||
} else if (!_address_sanitizers.empty()) {
|
||||
LOG(INFO) << "[Address Sanitizer] consumption is 0, but address sanitizers not empty. "
|
||||
<< ", mem tracker label: " << _label
|
||||
<< ", peak consumption: " << _consumption->peak_value()
|
||||
<< print_address_sanitizers();
|
||||
#endif
|
||||
}
|
||||
memory_memtrackerlimiter_cnt << -1;
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
void MemTrackerLimiter::add_address_sanitizers(void* buf, size_t size) {
|
||||
if (_type == Type::QUERY) {
|
||||
std::lock_guard<std::mutex> l(_address_sanitizers_mtx);
|
||||
auto it = _address_sanitizers.find(buf);
|
||||
if (it != _address_sanitizers.end()) {
|
||||
LOG(INFO) << "[Address Sanitizer] memory buf repeat add, mem tracker label: " << _label
|
||||
<< ", consumption: " << _consumption->current_value()
|
||||
<< ", peak consumption: " << _consumption->peak_value() << ", buf: " << buf
|
||||
<< ", size: " << size << ", old buf: " << it->first
|
||||
<< ", old size: " << it->second.size
|
||||
<< ", new stack_trace: " << get_stack_trace(1, "DISABLED")
|
||||
<< ", old stack_trace: " << it->second.stack_trace;
|
||||
}
|
||||
|
||||
// if alignment not equal to 0, maybe usable_size > size.
|
||||
AddressSanitizer as = {size, doris::config::enable_address_sanitizers_with_stack_trace
|
||||
? get_stack_trace(1, "DISABLED")
|
||||
: ""};
|
||||
_address_sanitizers.emplace(buf, as);
|
||||
}
|
||||
}
|
||||
|
||||
void MemTrackerLimiter::remove_address_sanitizers(void* buf, size_t size) {
|
||||
if (_type == Type::QUERY) {
|
||||
std::lock_guard<std::mutex> l(_address_sanitizers_mtx);
|
||||
auto it = _address_sanitizers.find(buf);
|
||||
if (it != _address_sanitizers.end()) {
|
||||
if (it->second.size != size) {
|
||||
LOG(INFO) << "[Address Sanitizer] free memory buf size inaccurate, mem tracker "
|
||||
"label: "
|
||||
<< _label << ", consumption: " << _consumption->current_value()
|
||||
<< ", peak consumption: " << _consumption->peak_value()
|
||||
<< ", buf: " << buf << ", size: " << size << ", old buf: " << it->first
|
||||
<< ", old size: " << it->second.size
|
||||
<< ", new stack_trace: " << get_stack_trace(1, "DISABLED")
|
||||
<< ", old stack_trace: " << it->second.stack_trace;
|
||||
}
|
||||
_address_sanitizers.erase(buf);
|
||||
} else {
|
||||
LOG(INFO) << "[Address Sanitizer] memory buf not exist, mem tracker label: " << _label
|
||||
<< ", consumption: " << _consumption->current_value()
|
||||
<< ", peak consumption: " << _consumption->peak_value() << ", buf: " << buf
|
||||
<< ", size: " << size << ", stack_trace: " << get_stack_trace(1, "DISABLED");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::string MemTrackerLimiter::print_address_sanitizers() {
|
||||
std::lock_guard<std::mutex> l(_address_sanitizers_mtx);
|
||||
std::string detail = "[Address Sanitizer]:";
|
||||
for (const auto& it : _address_sanitizers) {
|
||||
detail += fmt::format("\n {}, size {}, strack trace: {}", it.first, it.second.size,
|
||||
it.second.stack_trace);
|
||||
}
|
||||
return detail;
|
||||
}
|
||||
#endif
|
||||
|
||||
MemTracker::Snapshot MemTrackerLimiter::make_snapshot() const {
|
||||
Snapshot snapshot;
|
||||
snapshot.type = type_string(_type);
|
||||
|
||||
@ -205,6 +205,12 @@ public:
|
||||
// Log the memory usage when memory limit is exceeded.
|
||||
std::string tracker_limit_exceeded_str();
|
||||
|
||||
#ifndef NDEBUG
|
||||
void add_address_sanitizers(void* buf, size_t size);
|
||||
void remove_address_sanitizers(void* buf, size_t size);
|
||||
std::string print_address_sanitizers();
|
||||
#endif
|
||||
|
||||
std::string debug_string() override {
|
||||
std::stringstream msg;
|
||||
msg << "limit: " << _limit << "; "
|
||||
@ -245,6 +251,16 @@ private:
|
||||
// Avoid frequent printing.
|
||||
bool _enable_print_log_usage = false;
|
||||
static std::atomic<bool> _enable_print_log_process_usage;
|
||||
|
||||
#ifndef NDEBUG
|
||||
struct AddressSanitizer {
|
||||
size_t size;
|
||||
std::string stack_trace;
|
||||
};
|
||||
|
||||
std::mutex _address_sanitizers_mtx;
|
||||
std::unordered_map<void*, AddressSanitizer> _address_sanitizers;
|
||||
#endif
|
||||
};
|
||||
|
||||
inline int64_t MemTrackerLimiter::add_untracked_mem(int64_t bytes) {
|
||||
|
||||
@ -549,7 +549,7 @@ public:
|
||||
// must call create_thread_local_if_not_exits() before use thread_context().
|
||||
#define CONSUME_THREAD_MEM_TRACKER(size) \
|
||||
do { \
|
||||
if (doris::use_mem_hook || size == 0) { \
|
||||
if (size == 0 || doris::use_mem_hook) { \
|
||||
break; \
|
||||
} \
|
||||
if (doris::pthread_context_ptr_init) { \
|
||||
|
||||
@ -102,7 +102,7 @@ void _ingest_binlog(IngestBinlogArg* arg) {
|
||||
const auto& local_tablet_uid = local_tablet->tablet_uid();
|
||||
|
||||
std::shared_ptr<MemTrackerLimiter> mem_tracker = MemTrackerLimiter::create_shared(
|
||||
MemTrackerLimiter::Type::SCHEMA_CHANGE, fmt::format("IngestBinlog#TxnId={}", txn_id));
|
||||
MemTrackerLimiter::Type::OTHER, fmt::format("IngestBinlog#TxnId={}", txn_id));
|
||||
SCOPED_ATTACH_TASK(mem_tracker);
|
||||
|
||||
auto& request = arg->request;
|
||||
|
||||
@ -784,7 +784,7 @@ void PInternalServiceImpl::fetch_table_schema(google::protobuf::RpcController* c
|
||||
const TFileScanRangeParams& params = file_scan_range.params;
|
||||
|
||||
std::shared_ptr<MemTrackerLimiter> mem_tracker = MemTrackerLimiter::create_shared(
|
||||
MemTrackerLimiter::Type::SCHEMA_CHANGE,
|
||||
MemTrackerLimiter::Type::OTHER,
|
||||
fmt::format("{}#{}", params.format_type, params.file_type));
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(mem_tracker);
|
||||
|
||||
|
||||
@ -37,6 +37,7 @@
|
||||
#include "olap/tablet_schema.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/runtime_state.h"
|
||||
#include "runtime/thread_context.h"
|
||||
#include "util/key_util.h"
|
||||
#include "util/runtime_profile.h"
|
||||
#include "util/simd/bits.h"
|
||||
@ -166,7 +167,8 @@ void RowCache::erase(const RowCacheKey& key) {
|
||||
}
|
||||
|
||||
PointQueryExecutor::~PointQueryExecutor() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_mem_tracker);
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->point_query_executor_mem_tracker());
|
||||
_tablet.reset();
|
||||
_reusable.reset();
|
||||
_result_block.reset();
|
||||
@ -180,10 +182,7 @@ Status PointQueryExecutor::init(const PTabletKeyLookupRequest* request,
|
||||
// using cache
|
||||
__int128_t uuid =
|
||||
static_cast<__int128_t>(request->uuid().uuid_high()) << 64 | request->uuid().uuid_low();
|
||||
_mem_tracker = MemTrackerLimiter::create_shared(
|
||||
MemTrackerLimiter::Type::QUERY,
|
||||
fmt::format("PointQueryExecutor:{}#{}", uuid, request->tablet_id()));
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_mem_tracker);
|
||||
SCOPED_ATTACH_TASK(ExecEnv::GetInstance()->point_query_executor_mem_tracker());
|
||||
auto cache_handle = LookupConnectionCache::instance()->get(uuid);
|
||||
_binary_row_format = request->is_binary_row();
|
||||
_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(request->tablet_id());
|
||||
@ -234,7 +233,7 @@ Status PointQueryExecutor::init(const PTabletKeyLookupRequest* request,
|
||||
}
|
||||
|
||||
Status PointQueryExecutor::lookup_up() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_mem_tracker);
|
||||
SCOPED_ATTACH_TASK(ExecEnv::GetInstance()->point_query_executor_mem_tracker());
|
||||
RETURN_IF_ERROR(_lookup_row_key());
|
||||
RETURN_IF_ERROR(_lookup_row_data());
|
||||
RETURN_IF_ERROR(_output_data());
|
||||
|
||||
@ -323,7 +323,6 @@ private:
|
||||
std::vector<RowReadContext> _row_read_ctxs;
|
||||
std::shared_ptr<Reusable> _reusable;
|
||||
std::unique_ptr<vectorized::Block> _result_block;
|
||||
std::shared_ptr<MemTrackerLimiter> _mem_tracker;
|
||||
Metrics _profile_metrics;
|
||||
bool _binary_row_format = false;
|
||||
};
|
||||
|
||||
@ -53,6 +53,7 @@
|
||||
#include "gutil/endian.h"
|
||||
#include "gutil/strings/substitute.h"
|
||||
#include "orc/OrcFile.hh"
|
||||
#include "runtime/thread_context.h"
|
||||
#include "util/bit_util.h"
|
||||
#include "util/defer_op.h"
|
||||
#include "util/faststring.h"
|
||||
@ -99,10 +100,16 @@ private:
|
||||
ENABLE_FACTORY_CREATOR(Context);
|
||||
|
||||
public:
|
||||
Context() : ctx(nullptr) { buffer = std::make_unique<faststring>(); }
|
||||
Context() : ctx(nullptr) {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
buffer = std::make_unique<faststring>();
|
||||
}
|
||||
LZ4_stream_t* ctx;
|
||||
std::unique_ptr<faststring> buffer;
|
||||
~Context() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
if (ctx) {
|
||||
LZ4_freeStream(ctx);
|
||||
}
|
||||
@ -115,7 +122,11 @@ public:
|
||||
static Lz4BlockCompression s_instance;
|
||||
return &s_instance;
|
||||
}
|
||||
~Lz4BlockCompression() { _ctx_pool.clear(); }
|
||||
~Lz4BlockCompression() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
_ctx_pool.clear();
|
||||
}
|
||||
|
||||
Status compress(const Slice& input, faststring* output) override {
|
||||
if (input.size > INT_MAX) {
|
||||
@ -144,7 +155,13 @@ public:
|
||||
compressed_buf.size = max_len;
|
||||
} else {
|
||||
// reuse context buffer if max_len <= MAX_COMPRESSION_BUFFER_FOR_REUSE
|
||||
context->buffer->resize(max_len);
|
||||
{
|
||||
// context->buffer is resuable between queries, should accouting to
|
||||
// global tracker.
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
context->buffer->resize(max_len);
|
||||
}
|
||||
compressed_buf.data = reinterpret_cast<char*>(context->buffer->data());
|
||||
compressed_buf.size = max_len;
|
||||
}
|
||||
@ -253,10 +270,16 @@ private:
|
||||
ENABLE_FACTORY_CREATOR(CContext);
|
||||
|
||||
public:
|
||||
CContext() : ctx(nullptr) { buffer = std::make_unique<faststring>(); }
|
||||
CContext() : ctx(nullptr) {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
buffer = std::make_unique<faststring>();
|
||||
}
|
||||
LZ4F_compressionContext_t ctx;
|
||||
std::unique_ptr<faststring> buffer;
|
||||
~CContext() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
if (ctx) {
|
||||
LZ4F_freeCompressionContext(ctx);
|
||||
}
|
||||
@ -282,6 +305,8 @@ public:
|
||||
return &s_instance;
|
||||
}
|
||||
~Lz4fBlockCompression() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
_ctx_c_pool.clear();
|
||||
_ctx_d_pool.clear();
|
||||
}
|
||||
@ -326,8 +351,12 @@ private:
|
||||
compressed_buf.data = reinterpret_cast<char*>(output->data());
|
||||
compressed_buf.size = max_len;
|
||||
} else {
|
||||
// reuse context buffer if max_len <= MAX_COMPRESSION_BUFFER_FOR_REUSE
|
||||
context->buffer->resize(max_len);
|
||||
{
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
// reuse context buffer if max_len <= MAX_COMPRESSION_BUFFER_FOR_REUSE
|
||||
context->buffer->resize(max_len);
|
||||
}
|
||||
compressed_buf.data = reinterpret_cast<char*>(context->buffer->data());
|
||||
compressed_buf.size = max_len;
|
||||
}
|
||||
@ -482,10 +511,16 @@ private:
|
||||
ENABLE_FACTORY_CREATOR(Context);
|
||||
|
||||
public:
|
||||
Context() : ctx(nullptr) { buffer = std::make_unique<faststring>(); }
|
||||
Context() : ctx(nullptr) {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
buffer = std::make_unique<faststring>();
|
||||
}
|
||||
LZ4_streamHC_t* ctx;
|
||||
std::unique_ptr<faststring> buffer;
|
||||
~Context() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
if (ctx) {
|
||||
LZ4_freeStreamHC(ctx);
|
||||
}
|
||||
@ -498,7 +533,11 @@ public:
|
||||
static Lz4HCBlockCompression s_instance;
|
||||
return &s_instance;
|
||||
}
|
||||
~Lz4HCBlockCompression() { _ctx_pool.clear(); }
|
||||
~Lz4HCBlockCompression() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
_ctx_pool.clear();
|
||||
}
|
||||
|
||||
Status compress(const Slice& input, faststring* output) override {
|
||||
std::unique_ptr<Context> context;
|
||||
@ -519,7 +558,12 @@ public:
|
||||
compressed_buf.data = reinterpret_cast<char*>(output->data());
|
||||
compressed_buf.size = max_len;
|
||||
} else {
|
||||
context->buffer->resize(max_len);
|
||||
{
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
// reuse context buffer if max_len <= MAX_COMPRESSION_BUFFER_FOR_REUSE
|
||||
context->buffer->resize(max_len);
|
||||
}
|
||||
compressed_buf.data = reinterpret_cast<char*>(context->buffer->data());
|
||||
compressed_buf.size = max_len;
|
||||
}
|
||||
@ -781,10 +825,16 @@ private:
|
||||
ENABLE_FACTORY_CREATOR(CContext);
|
||||
|
||||
public:
|
||||
CContext() : ctx(nullptr) { buffer = std::make_unique<faststring>(); }
|
||||
CContext() : ctx(nullptr) {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
buffer = std::make_unique<faststring>();
|
||||
}
|
||||
ZSTD_CCtx* ctx;
|
||||
std::unique_ptr<faststring> buffer;
|
||||
~CContext() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
if (ctx) {
|
||||
ZSTD_freeCCtx(ctx);
|
||||
}
|
||||
@ -810,6 +860,8 @@ public:
|
||||
return &s_instance;
|
||||
}
|
||||
~ZstdBlockCompression() {
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
_ctx_c_pool.clear();
|
||||
_ctx_d_pool.clear();
|
||||
}
|
||||
@ -843,8 +895,12 @@ public:
|
||||
compressed_buf.data = reinterpret_cast<char*>(output->data());
|
||||
compressed_buf.size = max_len;
|
||||
} else {
|
||||
// reuse context buffer if max_len <= MAX_COMPRESSION_BUFFER_FOR_REUSE
|
||||
context->buffer->resize(max_len);
|
||||
{
|
||||
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(
|
||||
ExecEnv::GetInstance()->block_compression_mem_tracker());
|
||||
// reuse context buffer if max_len <= MAX_COMPRESSION_BUFFER_FOR_REUSE
|
||||
context->buffer->resize(max_len);
|
||||
}
|
||||
compressed_buf.data = reinterpret_cast<char*>(context->buffer->data());
|
||||
compressed_buf.size = max_len;
|
||||
}
|
||||
|
||||
@ -210,6 +210,30 @@ void Allocator<clear_memory_, mmap_populate, use_mmap, MemoryAllocator>::throw_b
|
||||
throw doris::Exception(doris::ErrorCode::MEM_ALLOC_FAILED, err);
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
template <bool clear_memory_, bool mmap_populate, bool use_mmap, typename MemoryAllocator>
|
||||
void Allocator<clear_memory_, mmap_populate, use_mmap, MemoryAllocator>::add_address_sanitizers(
|
||||
void* buf, size_t size) const {
|
||||
#ifdef BE_TEST
|
||||
if (!doris::ExecEnv::ready()) {
|
||||
return;
|
||||
}
|
||||
#endif
|
||||
doris::thread_context()->thread_mem_tracker()->add_address_sanitizers(buf, size);
|
||||
}
|
||||
|
||||
template <bool clear_memory_, bool mmap_populate, bool use_mmap, typename MemoryAllocator>
|
||||
void Allocator<clear_memory_, mmap_populate, use_mmap, MemoryAllocator>::remove_address_sanitizers(
|
||||
void* buf, size_t size) const {
|
||||
#ifdef BE_TEST
|
||||
if (!doris::ExecEnv::ready()) {
|
||||
return;
|
||||
}
|
||||
#endif
|
||||
doris::thread_context()->thread_mem_tracker()->remove_address_sanitizers(buf, size);
|
||||
}
|
||||
#endif
|
||||
|
||||
template <bool clear_memory_, bool mmap_populate, bool use_mmap, typename MemoryAllocator>
|
||||
void* Allocator<clear_memory_, mmap_populate, use_mmap, MemoryAllocator>::alloc(size_t size,
|
||||
size_t alignment) {
|
||||
|
||||
@ -231,6 +231,10 @@ public:
|
||||
void consume_memory(size_t size) const;
|
||||
void release_memory(size_t size) const;
|
||||
void throw_bad_alloc(const std::string& err) const;
|
||||
#ifndef NDEBUG
|
||||
void add_address_sanitizers(void* buf, size_t size) const;
|
||||
void remove_address_sanitizers(void* buf, size_t size) const;
|
||||
#endif
|
||||
|
||||
void* alloc(size_t size, size_t alignment = 0);
|
||||
void* realloc(void* buf, size_t old_size, size_t new_size, size_t alignment = 0);
|
||||
@ -238,6 +242,7 @@ public:
|
||||
/// Allocate memory range.
|
||||
void* alloc_impl(size_t size, size_t alignment = 0) {
|
||||
memory_check(size);
|
||||
// consume memory in tracker before alloc, similar to early declaration.
|
||||
consume_memory(size);
|
||||
void* buf;
|
||||
size_t record_size = size;
|
||||
@ -273,6 +278,9 @@ public:
|
||||
if constexpr (MemoryAllocator::need_record_actual_size()) {
|
||||
record_size = MemoryAllocator::allocated_size(buf);
|
||||
}
|
||||
#ifndef NDEBUG
|
||||
add_address_sanitizers(buf, size);
|
||||
#endif
|
||||
} else {
|
||||
buf = nullptr;
|
||||
int res = MemoryAllocator::posix_memalign(&buf, alignment, size);
|
||||
@ -282,6 +290,9 @@ public:
|
||||
throw_bad_alloc(
|
||||
fmt::format("Cannot allocate memory (posix_memalign) {}.", size));
|
||||
}
|
||||
#ifndef NDEBUG
|
||||
add_address_sanitizers(buf, size);
|
||||
#endif
|
||||
|
||||
if constexpr (clear_memory) memset(buf, 0, size);
|
||||
|
||||
@ -303,6 +314,9 @@ public:
|
||||
throw_bad_alloc(fmt::format("Allocator: Cannot munmap {}.", size));
|
||||
}
|
||||
} else {
|
||||
#ifndef NDEBUG
|
||||
remove_address_sanitizers(buf, size);
|
||||
#endif
|
||||
MemoryAllocator::free(buf);
|
||||
}
|
||||
release_memory(size);
|
||||
@ -326,6 +340,9 @@ public:
|
||||
if (!use_mmap ||
|
||||
(old_size < doris::config::mmap_threshold && new_size < doris::config::mmap_threshold &&
|
||||
alignment <= MALLOC_MIN_ALIGNMENT)) {
|
||||
#ifndef NDEBUG
|
||||
remove_address_sanitizers(buf, old_size);
|
||||
#endif
|
||||
/// Resize malloc'd memory region with no special alignment requirement.
|
||||
void* new_buf = MemoryAllocator::realloc(buf, new_size);
|
||||
if (nullptr == new_buf) {
|
||||
@ -333,6 +350,10 @@ public:
|
||||
throw_bad_alloc(fmt::format("Allocator: Cannot realloc from {} to {}.", old_size,
|
||||
new_size));
|
||||
}
|
||||
#ifndef NDEBUG
|
||||
add_address_sanitizers(
|
||||
new_buf, new_size); // usually, buf addr = new_buf addr, asan maybe not equal.
|
||||
#endif
|
||||
|
||||
buf = new_buf;
|
||||
if constexpr (clear_memory)
|
||||
@ -362,6 +383,10 @@ public:
|
||||
// Big allocs that requires a copy.
|
||||
void* new_buf = alloc(new_size, alignment);
|
||||
memcpy(new_buf, buf, std::min(old_size, new_size));
|
||||
#ifndef NDEBUG
|
||||
add_address_sanitizers(new_buf, new_size);
|
||||
remove_address_sanitizers(buf, old_size);
|
||||
#endif
|
||||
free(buf, old_size);
|
||||
buf = new_buf;
|
||||
}
|
||||
|
||||
@ -36,9 +36,12 @@ template <typename T, size_t initial_bytes = 4096, typename TAllocator = Allocat
|
||||
size_t pad_right_ = 0, size_t pad_left_ = 0>
|
||||
class PODArray;
|
||||
|
||||
/** For columns. Padding is enough to read and write xmm-register at the address of the last element. */
|
||||
/** For columns. Padding is enough to read and write xmm-register at the address of the last element.
|
||||
* TODO, pad_right is temporarily changed from 15 to 16, will waste 1 bytes,
|
||||
* can rollback after fix wrong reinterpret_cast column and PODArray swap.
|
||||
*/
|
||||
template <typename T, size_t initial_bytes = 4096, typename TAllocator = Allocator<false>>
|
||||
using PaddedPODArray = PODArray<T, initial_bytes, TAllocator, 15, 16>;
|
||||
using PaddedPODArray = PODArray<T, initial_bytes, TAllocator, 16, 16>;
|
||||
|
||||
/** A helper for declaring PODArray that uses inline memory.
|
||||
* The initial size is set to use all the inline bytes, since using less would
|
||||
|
||||
@ -189,7 +189,7 @@ else
|
||||
fi
|
||||
|
||||
## set asan and ubsan env to generate core file
|
||||
export ASAN_OPTIONS=symbolize=1:abort_on_error=1:disable_coredump=0:unmap_shadow_on_exit=1:detect_container_overflow=0
|
||||
export ASAN_OPTIONS=symbolize=1:abort_on_error=1:disable_coredump=0:unmap_shadow_on_exit=1:detect_container_overflow=0:check_malloc_usable_size=0
|
||||
export UBSAN_OPTIONS=print_stacktrace=1
|
||||
|
||||
## set TCMALLOC_HEAP_LIMIT_MB to limit memory used by tcmalloc
|
||||
|
||||
@ -267,7 +267,8 @@ fi
|
||||
export AWS_MAX_ATTEMPTS=2
|
||||
|
||||
## set asan and ubsan env to generate core file
|
||||
export ASAN_OPTIONS=symbolize=1:abort_on_error=1:disable_coredump=0:unmap_shadow_on_exit=1:detect_container_overflow=0
|
||||
## detect_container_overflow=0, https://github.com/google/sanitizers/issues/193
|
||||
export ASAN_OPTIONS=symbolize=1:abort_on_error=1:disable_coredump=0:unmap_shadow_on_exit=1:detect_container_overflow=0:check_malloc_usable_size=0
|
||||
export UBSAN_OPTIONS=print_stacktrace=1
|
||||
|
||||
## set TCMALLOC_HEAP_LIMIT_MB to limit memory used by tcmalloc
|
||||
|
||||
@ -405,7 +405,8 @@ export ORC_EXAMPLE_DIR="${DORIS_HOME}/be/src/apache-orc/examples"
|
||||
|
||||
# set asan and ubsan env to generate core file
|
||||
export DORIS_HOME="${DORIS_TEST_BINARY_DIR}/"
|
||||
export ASAN_OPTIONS=symbolize=1:abort_on_error=1:disable_coredump=0:unmap_shadow_on_exit=1:detect_container_overflow=0
|
||||
## detect_container_overflow=0, https://github.com/google/sanitizers/issues/193
|
||||
export ASAN_OPTIONS=symbolize=1:abort_on_error=1:disable_coredump=0:unmap_shadow_on_exit=1:detect_container_overflow=0:check_malloc_usable_size=0
|
||||
export UBSAN_OPTIONS=print_stacktrace=1
|
||||
export JAVA_OPTS="-Xmx1024m -DlogPath=${DORIS_HOME}/log/jni.log -Xloggc:${DORIS_HOME}/log/be.gc.log.${CUR_DATE} -Dsun.java.command=DorisBE -XX:-CriticalJNINatives -DJDBC_MIN_POOL=1 -DJDBC_MAX_POOL=100 -DJDBC_MAX_IDLE_TIME=300000"
|
||||
|
||||
|
||||
Reference in New Issue
Block a user