Browse Source

fix(profiler): collect records when thread exit

GitOrigin-RevId: fb796dd61c
release-1.6
Megvii Engine Team 4 years ago
parent
commit
59d5976622
8 changed files with 148 additions and 127 deletions
  1. +6
    -5
      imperative/python/megengine/tools/profiler.py
  2. +72
    -78
      imperative/src/impl/interpreter/interpreter_impl.cpp
  3. +5
    -0
      imperative/src/impl/physical_tensor.cpp
  4. +7
    -3
      imperative/src/impl/profiler.cpp
  5. +25
    -9
      imperative/src/impl/profiler/chrome_timeline.cpp
  6. +3
    -1
      imperative/src/impl/profiler/states.h
  7. +21
    -17
      imperative/src/impl/profiler_plugin.cpp
  8. +9
    -14
      imperative/src/include/megbrain/imperative/profiler.h

+ 6
- 5
imperative/python/megengine/tools/profiler.py View File

@@ -53,10 +53,6 @@ def main():
if args.format:
prof_args["formats"] = args.format

if args.clean:
for file in os.listdir(profiler.directory):
os.remove(os.path.join(profiler.directory, file))

if len(extras) == 0:
if not args.merge_trace_events:
parser.print_usage()
@@ -74,9 +70,14 @@ def main():
sys.argv[:] = [filename, *extras[1:]]

profiler = Profiler(**prof_args)

if args.clean:
for file in os.listdir(profiler.directory):
os.remove(os.path.join(profiler.directory, file))

with profiler:
if args.module:
runpy.run_module(filename)
run_module(filename)
else:
run_script(filename)
profiler.dump()


+ 72
- 78
imperative/src/impl/interpreter/interpreter_impl.cpp View File

@@ -30,12 +30,6 @@ using namespace imperative;
using namespace interpreter;
using namespace interpreter::intl;

#define RECORD_EVENT(type, ...) \
if (Profiler::is_profiling()) { \
Profiler::record<type>(type{__VA_ARGS__}); \
} \


namespace {
auto tinfo_to_tid(SmallVector<TensorInfo*> tinfo) {
SmallVector<uint64_t> tid;
@@ -70,12 +64,12 @@ namespace mgb {
**/
SYMBOL_EXPORT
void imperative_log_profile_begin(const char* message) {
RECORD_EVENT(CustomEvent, std::string{message});
MGB_RECORD_EVENT(CustomEvent, std::string{message});
}

SYMBOL_EXPORT
void imperative_log_profile_end(const char* message) {
RECORD_EVENT(CustomFinishEvent, std::string{message});
MGB_RECORD_EVENT(CustomFinishEvent, std::string{message});
}

SYMBOL_EXPORT
@@ -159,13 +153,13 @@ TensorInfo* ChannelImpl::put_impl(const DeviceTensorND& data, const HostTensorND
auto& state = get_channel_state();
auto _ = StackManager::Guard{"Put", &state.stack_manager};
auto info = alloc();
RECORD_EVENT(TensorCommandEvent, info->id, TensorCommandKind::Put);
MGB_RECORD_EVENT(TensorCommandEvent, info->id, TensorCommandKind::Put);
init(info, {data.layout(), data.comp_node()});
info->mem_desc.id = StorageIdentifier::make(++m_storage_id);
info->ptr = Tensor::make(data, hvalue);
RECORD_EVENT(TensorProduceEvent, info->id, info->desc.layout, info->desc.comp_node, data.raw_ptr());
MGB_RECORD_EVENT(TensorProduceEvent, info->id, info->desc.layout, info->desc.comp_node, data.raw_ptr());
info->status = TensorInfo::Produced;
RECORD_EVENT(TensorCommandFinishEvent, info->id, TensorCommandKind::Put);
MGB_RECORD_EVENT(TensorCommandFinishEvent, info->id, TensorCommandKind::Put);
return info;
}

@@ -231,7 +225,7 @@ void ChannelImpl::dispatch_default_cpu(
auto _ = StackManager::Guard(name, &state.stack_manager);

auto [output_descs, validated] = OpDef::infer_output_attrs_fallible(*op, input_descs);
RECORD_EVENT(ShapeInferEvent, validated);
MGB_RECORD_EVENT(ShapeInferEvent, validated);

SmallVector<DeviceTensorND> input_tensornds;
input_tensornds.reserve(input_descs.size());
@@ -289,7 +283,7 @@ void ChannelImpl::dispatch_default_cpu(
}
return op_info;
};
RECORD_EVENT(OpDispatchEvent, op_id, op->trait()->name, op_info_getter,
MGB_RECORD_EVENT(OpDispatchEvent, op_id, name, op_info_getter,
tinfo_to_tid(input_infos), tinfo_to_tid(output_infos),
state.stack_manager.dump());
}
@@ -306,7 +300,7 @@ void ChannelImpl::dispatch_kernel(
auto _ = StackManager::Guard{name, &state.stack_manager};

auto [output_descs, validated] = OpDef::infer_output_attrs_fallible(*op, input_descs);
RECORD_EVENT(ShapeInferEvent, validated);
MGB_RECORD_EVENT(ShapeInferEvent, validated);

ApplyOp cmd{Profiler::next_id(), std::move(op)};
cmd.inputs = std::move(input_infos);
@@ -332,7 +326,7 @@ void ChannelImpl::dispatch_kernel(
}
return op_info;
};
RECORD_EVENT(OpDispatchEvent, cmd.id, cmd.op->trait()->name, op_info_getter,
MGB_RECORD_EVENT(OpDispatchEvent, cmd.id, name, op_info_getter,
tinfo_to_tid(cmd.inputs), tinfo_to_tid(cmd.outputs),
state.stack_manager.dump());
m_buffer.enqueue(std::move(cmd));
@@ -426,7 +420,7 @@ DType ChannelImpl::get_dtype(Handle handle) {
mgb_assert(m_valid_handle.find(handle) != m_valid_handle.end(),
"invalid handle: %p", handle);
auto info = reinterpret_cast<TensorInfo*>(handle);
RECORD_EVENT(TensorGetPropEvent, info->id, TensorProp::DType);
MGB_RECORD_EVENT(TensorGetPropEvent, info->id, TensorProp::DType);
auto ret = info->desc.layout.dtype;
mgb_assert(ret.valid());
return ret;
@@ -438,7 +432,7 @@ CompNode ChannelImpl::get_device(Handle handle) {
mgb_assert(m_valid_handle.find(handle) != m_valid_handle.end(),
"invalid handle: %p", handle);
auto info = reinterpret_cast<TensorInfo*>(handle);
RECORD_EVENT(TensorGetPropEvent, info->id, TensorProp::Device);
MGB_RECORD_EVENT(TensorGetPropEvent, info->id, TensorProp::Device);
auto ret = info->desc.comp_node;
mgb_assert(ret.valid());
return ret;
@@ -512,7 +506,7 @@ TensorInfo* ChannelImpl::alloc() {

void ChannelImpl::init(TensorInfo* info, LogicalTensorDesc desc) {
m_valid_handle.insert(info);
RECORD_EVENT(TensorDeclareEvent, info->id, info->name);
MGB_RECORD_EVENT(TensorDeclareEvent, info->id, info->name);
info->status = TensorInfo::Allocated;
info->desc = std::move(desc);
info->mem_desc.layout = info->desc.layout;
@@ -554,7 +548,7 @@ void ChannelImpl::free(TensorInfo* ptr) {
}

void ChannelImpl::recursive_free(TensorInfo* ptr) {
RECORD_EVENT(TensorCommandEvent, ptr->id, TensorCommandKind::RecFree);
MGB_RECORD_EVENT(TensorCommandEvent, ptr->id, TensorCommandKind::RecFree);
SmallVector<TensorInfo*> inps;
if (ptr->producer) {
for (auto i : ptr->producer->inputs) {
@@ -569,7 +563,7 @@ void ChannelImpl::recursive_free(TensorInfo* ptr) {
recursive_free(i);
}
}
RECORD_EVENT(TensorCommandFinishEvent, ptr->id, TensorCommandKind::RecFree);
MGB_RECORD_EVENT(TensorCommandFinishEvent, ptr->id, TensorCommandKind::RecFree);
}

void ChannelImpl::real_free(TensorInfo* ptr) {
@@ -581,9 +575,9 @@ void ChannelImpl::real_free(TensorInfo* ptr) {
ptr->detach_producer();
bool has_value = ptr->ptr != nullptr;
if (has_value) {
RECORD_EVENT(TensorReleaseEvent, ptr->id);
MGB_RECORD_EVENT(TensorReleaseEvent, ptr->id);
}
RECORD_EVENT(TensorEraseEvent, ptr->id, ptr->ptr_use_count);
MGB_RECORD_EVENT(TensorEraseEvent, ptr->id, ptr->ptr_use_count);
ptr->status = TensorInfo::Deleted;
MGB_LOCK_GUARD(m_mutex);
m_pool.free(ptr);
@@ -599,7 +593,7 @@ void ChannelImpl::produce_tensor(TensorInfo* dest, TensorPtr ptr) {
auto& state = get_worker_state();
MGB_LOCK_GUARD(m_mutex);
m_dtr.update_used_time(dest);
RECORD_EVENT(TensorProduceEvent, dest->id, ptr->layout(), ptr->comp_node(), ptr->dev_tensor().raw_ptr());
MGB_RECORD_EVENT(TensorProduceEvent, dest->id, ptr->layout(), ptr->comp_node(), ptr->dev_tensor().raw_ptr());
// update tensor desc for static infer
dest->desc.layout = ptr->layout();
dest->desc.comp_node = ptr->comp_node();
@@ -614,7 +608,7 @@ void ChannelImpl::produce_tensor(TensorInfo* dest, TensorPtr ptr) {
}

void ChannelImpl::release_tensor(TensorInfo* dest) {
RECORD_EVENT(TensorReleaseEvent, dest->id);
MGB_RECORD_EVENT(TensorReleaseEvent, dest->id);
MGB_LOCK_GUARD(m_mutex);
dest->ptr.reset();
}
@@ -625,9 +619,9 @@ void ChannelImpl::regenerate(TensorInfo* dest) {
m_apply_stack.push({ApplyOp{path->id, path->op, path->inputs, path->outputs, {}}, 0, dest});
if (!m_applying) flush_apply_stack();
} else if (dest->evict_type == EvictType::SWAP) {
RECORD_EVENT(TensorCommandEvent, dest->id, TensorCommandKind::ReGen);
MGB_RECORD_EVENT(TensorCommandEvent, dest->id, TensorCommandKind::ReGen);
produce_tensor(dest, Tensor::make(dest->h_value));
RECORD_EVENT(TensorCommandFinishEvent, dest->id, TensorCommandKind::ReGen);
MGB_RECORD_EVENT(TensorCommandFinishEvent, dest->id, TensorCommandKind::ReGen);
}
}

@@ -695,7 +689,7 @@ void ChannelImpl::do_apply_op(const ApplyOp& cmd) {
}
return outputs;
};
RECORD_EVENT(OpExecuteEvent, apply_id);
MGB_RECORD_EVENT(OpExecuteEvent, apply_id);
// Begin profiling operator
SmallVector<std::pair<CompNode, uint64_t>> kernels;
if (profiling_device) {
@@ -710,9 +704,9 @@ void ChannelImpl::do_apply_op(const ApplyOp& cmd) {
}
for (auto* input: cmd.inputs) {
auto input_id = input->id;
RECORD_EVENT(OpInputEvent, input_id);
RECORD_EVENT(TensorUsageEvent, input_id);
RECORD_EVENT(OpInputFinishEvent, input_id);
MGB_RECORD_EVENT(OpInputEvent, input_id);
MGB_RECORD_EVENT(TensorUsageEvent, input_id);
MGB_RECORD_EVENT(OpInputFinishEvent, input_id);
}
// Fused by command buffer. @see: CommandBuffer::fuse_del
// Now if dest is inplacable, it's refcnt would be decreased to 1 and owned by tensor_inputs after Del.
@@ -721,40 +715,40 @@ void ChannelImpl::do_apply_op(const ApplyOp& cmd) {
// refcnt --, owners: [tensor_inputs]
// if it's decreased to 1, would be detected at @see: proxy_graph_detail::apply_on_physical_tensor
uint64_t del_id = del->id;
RECORD_EVENT(TensorCommandEvent, del_id, TensorCommandKind::Del);
MGB_RECORD_EVENT(TensorCommandEvent, del_id, TensorCommandKind::Del);
free(del);
RECORD_EVENT(TensorCommandFinishEvent, del_id, TensorCommandKind::Del);
MGB_RECORD_EVENT(TensorCommandFinishEvent, del_id, TensorCommandKind::Del);
}
// Before wait
//TODO: split operator wait and execute so that OpWait could be corrected recorded.
// Before execute
for (auto&& [device, kernel_id]: kernels) {
RECORD_EVENT(KernelLaunchEvent, apply_id, kernel_id, device);
RECORD_EVENT(RecordDeviceEvent, Timer::record_device(device));
MGB_RECORD_EVENT(KernelLaunchEvent, apply_id, kernel_id, device);
MGB_RECORD_EVENT(RecordDeviceEvent, Timer::record_device(device));
}
// Apply op
// Here std::move is REQUIRED for removing duplicated references.
auto outputs = apply_on_physical_tensor(apply_on_physical_tensor, *cmd.op, inputs);
// After execute
for (auto&& [device, kernel_id]: kernels) {
RECORD_EVENT(RecordDeviceEvent, Timer::record_device(device));
RECORD_EVENT(KernelLaunchFinishEvent, apply_id, kernel_id, device);
MGB_RECORD_EVENT(RecordDeviceEvent, Timer::record_device(device));
MGB_RECORD_EVENT(KernelLaunchFinishEvent, apply_id, kernel_id, device);
}
// End profiling operator
mgb_assert(outputs.size() == cmd.outputs.size());
for (size_t i = 0; i < outputs.size(); ++i) {
auto output = cmd.outputs[i];
if (output == nullptr) {
RECORD_EVENT(OpOutputEvent, 0);
RECORD_EVENT(OpOutputFinishEvent, 0);
MGB_RECORD_EVENT(OpOutputEvent, 0);
MGB_RECORD_EVENT(OpOutputFinishEvent, 0);
} else if (output->ptr != nullptr) {
RECORD_EVENT(OpOutputEvent, output->id);
RECORD_EVENT(OpOutputFinishEvent, output->id);
MGB_RECORD_EVENT(OpOutputEvent, output->id);
MGB_RECORD_EVENT(OpOutputFinishEvent, output->id);
} else {
RECORD_EVENT(OpOutputEvent, output->id);
MGB_RECORD_EVENT(OpOutputEvent, output->id);
produce_tensor(output, outputs[i].tensor);
output->mem_desc = outputs[i].desc;
RECORD_EVENT(OpOutputFinishEvent, output->id);
MGB_RECORD_EVENT(OpOutputFinishEvent, output->id);
sample_on_device(output->desc.comp_node, false);
}
}
@@ -775,7 +769,7 @@ void ChannelImpl::do_apply_op(const ApplyOp& cmd) {
}
m_dtr.unpin(cmd.inputs);
}
RECORD_EVENT(OpExecuteFinishEvent, apply_id);
MGB_RECORD_EVENT(OpExecuteFinishEvent, apply_id);
// End profiling operator
}

@@ -789,7 +783,7 @@ void ChannelImpl::flush_apply_stack() {
m_dtr.pin(cmd.inputs);
}
if (recomp) {
RECORD_EVENT(TensorCommandEvent, recomp->id, TensorCommandKind::ReGen);
MGB_RECORD_EVENT(TensorCommandEvent, recomp->id, TensorCommandKind::ReGen);
}
}
bool regen = false;
@@ -812,7 +806,7 @@ void ChannelImpl::flush_apply_stack() {
m_apply_stack.pop();
do_apply_op(cmd_backup);
if (recomp_backup) {
RECORD_EVENT(TensorCommandFinishEvent, recomp_backup->id, TensorCommandKind::ReGen);
MGB_RECORD_EVENT(TensorCommandFinishEvent, recomp_backup->id, TensorCommandKind::ReGen);
for (auto o : cmd_backup.outputs) {
if (o) {
m_dtr.update_dsu_after_recompute(o);
@@ -831,7 +825,7 @@ bool ChannelImpl::auto_evict(size_t force_num) {
size_t current_memory = m_dtr.comp_node.get_used_memory();
size_t flag = false;
while ((state.options.dtr_eviction_threshold > 0 && current_memory > state.options.dtr_eviction_threshold) || force_num > 0) {
RECORD_EVENT(AutoEvictEvent);
MGB_RECORD_EVENT(AutoEvictEvent);
sample_on_device(m_dtr.comp_node, false);
auto best = m_dtr.find_best_tensor(state.options.enable_dtr_sqrt_sampling && !force_num);
if (!best) {
@@ -849,7 +843,7 @@ bool ChannelImpl::auto_evict(size_t force_num) {
m_dtr.update_dsu_after_evict(best);
}
sample_on_device(m_dtr.comp_node, false);
RECORD_EVENT(AutoEvictFinishEvent);
MGB_RECORD_EVENT(AutoEvictFinishEvent);
}
return flag;
}
@@ -888,7 +882,7 @@ TensorPtr ChannelImpl::wait_tensor(TensorInfo* info, TensorProp prop) {
mgb_assert(!m_waitee, "duplicate waitee");
m_waitee = info;
m_waitee_id = Profiler::next_id();
RECORD_EVENT(TensorWaitPropEvent, info->id, m_waitee_id, prop);
MGB_RECORD_EVENT(TensorWaitPropEvent, info->id, m_waitee_id, prop);
bool require_host = prop == TensorProp::HostValue;
auto host_available = [&]{
return info->ptr && info->ptr->value_fetched();
@@ -904,14 +898,14 @@ TensorPtr ChannelImpl::wait_tensor(TensorInfo* info, TensorProp prop) {
check_worker_exc_unsafe();
return require_host ? host_available() : static_cast<bool>(info->ptr);
});
RECORD_EVENT(TensorWaitPropFinishEvent, info->id, m_waitee_id, prop);
MGB_RECORD_EVENT(TensorWaitPropFinishEvent, info->id, m_waitee_id, prop);
m_waitee = nullptr;
return info->ptr;
}

void ChannelImpl::notify_tensor_unsafe(TensorInfo* info) {
if (info == m_waitee) {
RECORD_EVENT(TensorNotifyPropEvent, info->id);
MGB_RECORD_EVENT(TensorNotifyPropEvent, info->id);
m_cv.notify_all();
}
}
@@ -1014,12 +1008,12 @@ void ChannelImpl::process_one_task(Command& icmd) {
auto cmd_visitor = [&](const auto& cmd) {
using T = std::decay_t<decltype(cmd)>;
if constexpr (std::is_same_v<T, Put>) {
RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::Put);
RECORD_EVENT(RecordDeviceEvent, Timer::record_device(cmd.value.comp_node()));
MGB_RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::Put);
MGB_RECORD_EVENT(RecordDeviceEvent, Timer::record_device(cmd.value.comp_node()));
auto value = cmd.no_cache ? std::make_shared<Tensor>(cmd.value) : Tensor::make(cmd.value);
RECORD_EVENT(RecordDeviceEvent, Timer::record_device(cmd.value.comp_node()));
MGB_RECORD_EVENT(RecordDeviceEvent, Timer::record_device(cmd.value.comp_node()));
produce_tensor(cmd.dest, std::move(value));
RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::Put);
MGB_RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::Put);
sample_on_device(cmd.dest->desc.comp_node, false);
} else if constexpr (std::is_same_v<T, ApplyOp>) {
for (auto& i : cmd.inputs) {
@@ -1088,11 +1082,11 @@ void ChannelImpl::process_one_task(Command& icmd) {
}
}
} else if constexpr (std::is_same_v<T, Del>) {
RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::Del);
MGB_RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::Del);
CompNode device = cmd.dest->desc.comp_node;
uint64_t tensor_id = cmd.dest->id;
free(cmd.dest);
RECORD_EVENT(TensorCommandFinishEvent, tensor_id, TensorCommandKind::Del);
MGB_RECORD_EVENT(TensorCommandFinishEvent, tensor_id, TensorCommandKind::Del);
sample_on_device(device, false);
} else if constexpr (std::is_same_v<T, GetValue>) {
if (cmd.dest->invalid) return;
@@ -1106,64 +1100,64 @@ void ChannelImpl::process_one_task(Command& icmd) {
imperative_log_profile_end("GetValue");
} else if constexpr (std::is_same_v<T, SwapIn>) {
if (cmd.dest->invalid) return;
RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::SwapIn);
MGB_RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::SwapIn);
produce_tensor(cmd.dest, Tensor::make(cmd.dest->h_value));
RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::SwapIn);
MGB_RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::SwapIn);
sample_on_device(cmd.dest->desc.comp_node, false);
} else if constexpr (std::is_same_v<T, SwapOut>) {
if (cmd.dest->invalid) return;
RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::SwapOut);
MGB_RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::SwapOut);
cmd.dest->h_value = cmd.dest->ptr->get_value();
if (cmd.dest->evict_type == EvictType::NONE) {
cmd.dest->evict_type = EvictType::SWAP;
cmd.dest->status = TensorInfo::Swapped;
release_tensor(cmd.dest);
}
RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::SwapOut);
MGB_RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::SwapOut);
sample_on_device(cmd.dest->desc.comp_node, false);
} else if constexpr (std::is_same_v<T, Drop>) {
if (cmd.dest->invalid) return;
RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::Drop);
MGB_RECORD_EVENT(TensorCommandEvent, cmd.dest->id, TensorCommandKind::Drop);
do_drop(cmd.dest, true);
RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::Drop);
MGB_RECORD_EVENT(TensorCommandFinishEvent, cmd.dest->id, TensorCommandKind::Drop);
} else if constexpr (std::is_same_v<T, SetOption>) {
options.set_option(cmd.key, cmd.value);
} else if constexpr (std::is_same_v<T, StartProfile>) {
RECORD_EVENT(StartProfileEvent);
MGB_RECORD_EVENT(StartProfileEvent);
CompNode::sync_all();
for (auto* info: cmd.capture_tensors) {
RECORD_EVENT(TensorDeclareEvent, info->id, info->name);
MGB_RECORD_EVENT(TensorDeclareEvent, info->id, info->name);
if (info->status == TensorInfo::Produced) {
// TODO: handle swap/drop
RECORD_EVENT(TensorProduceEvent, info->id, info->desc.layout, info->desc.comp_node, info->ptr->dev_tensor().raw_ptr());
MGB_RECORD_EVENT(TensorProduceEvent, info->id, info->desc.layout, info->desc.comp_node, info->ptr->dev_tensor().raw_ptr());
}
}
CompNode::foreach([&](CompNode device){
if (Profiler::get_option("sample_rate", 0)) {
sample_on_device(device, true);
}
RECORD_EVENT(RecordDeviceEvent, Timer::record_device(device));
MGB_RECORD_EVENT(RecordDeviceEvent, Timer::record_device(device));
});
RECORD_EVENT(StartProfileFinishEvent);
MGB_RECORD_EVENT(StartProfileFinishEvent);
} else if constexpr (std::is_same_v<T, StopProfile>) {
RECORD_EVENT(StopProfileEvent);
MGB_RECORD_EVENT(StopProfileEvent);
for (auto* info: cmd.escape_tensors) {
bool has_value = info->status == TensorInfo::Produced;
if (has_value) {
RECORD_EVENT(TensorReleaseEvent, info->id);
MGB_RECORD_EVENT(TensorReleaseEvent, info->id);
}
RECORD_EVENT(TensorEraseEvent, info->id);
MGB_RECORD_EVENT(TensorEraseEvent, info->id);
}
CompNode::foreach([&](CompNode device){
if (Profiler::get_option("sample_rate", 0)) {
sample_on_device(device, true);
}
});
RECORD_EVENT(StopProfileFinishEvent);
MGB_RECORD_EVENT(StopProfileFinishEvent);
} else if constexpr (std::is_same_v<T, PushScope>) {
RECORD_EVENT(ScopeEvent, cmd.scope_name);
MGB_RECORD_EVENT(ScopeEvent, cmd.scope_name);
} else if constexpr (std::is_same_v<T, PopScope>) {
RECORD_EVENT(ScopeFinishEvent, cmd.scope_name);
MGB_RECORD_EVENT(ScopeFinishEvent, cmd.scope_name);
} else {
static_assert(!std::is_same_v<T, T>);
}
@@ -1186,7 +1180,7 @@ void ChannelImpl::process_one_task(Command& icmd) {
cmd.dest->invalid = true;
}
m_worker_exc = std::current_exception();
RECORD_EVENT(WorkerExceptionEvent);
MGB_RECORD_EVENT(WorkerExceptionEvent);
if (m_waitee) {
notify_tensor_unsafe(m_waitee);
}
@@ -1347,7 +1341,7 @@ void ChannelImpl::push_scope(std::string name) {
mgb_assert(check_available(), "Channel already closed");
auto& state = get_channel_state();
state.stack_manager.enter(name);
RECORD_EVENT(ScopeEvent, name);
MGB_RECORD_EVENT(ScopeEvent, name);
m_buffer.enqueue(PushScope{name});
}

@@ -1356,7 +1350,7 @@ void ChannelImpl::pop_scope(std::string name) {
mgb_assert(check_available(), "Channel already closed");
auto& state = get_channel_state();
state.stack_manager.exit(name);
RECORD_EVENT(ScopeFinishEvent, name);
MGB_RECORD_EVENT(ScopeFinishEvent, name);
m_buffer.enqueue(PopScope{name});
}

@@ -1376,9 +1370,9 @@ void ChannelImpl::sample_on_device(CompNode device, bool force) {
return;
}
}
RECORD_EVENT(SampleDeviceEvent, device);
MGB_RECORD_EVENT(SampleDeviceEvent, device);
auto [total, free] = device.get_mem_status_bytes();
RECORD_EVENT(SampleDeviceFinishEvent, device, total, free);
MGB_RECORD_EVENT(SampleDeviceFinishEvent, device, total, free);
}

void ChannelImpl::DynamicSublinear::pin(const SmallVector<TensorInfo*>& vec) {


+ 5
- 0
imperative/src/impl/physical_tensor.cpp View File

@@ -11,10 +11,13 @@

#include "megbrain/imperative.h"
#include "megbrain/imperative/blob_manager.h"
#include "megbrain/imperative/profiler.h"

#include "./event_pool.h"
#include "./async_releaser.h"

#include "./profiler/events.h"

#include <mutex>

namespace mgb {
@@ -128,9 +131,11 @@ Tensor::Tensor(BlobPtr blob, const TensorLayout& layout, size_t offset, const Ho
Tensor::Tensor(const HostTensorND &hv)
: Tensor(hv.layout(), hv.comp_node()) {
m_value = hv;
MGB_RECORD_EVENT(profiler::HostToDeviceEvent, hv.layout(), hv.comp_node(), hv.raw_ptr(), dev_tensor().raw_ptr());
dev_tensor().copy_from_fixlayout(hv);
// even though hv is saved in m_value, Tensor itself could be
// released before copy completes
MGB_RECORD_EVENT(profiler::HostToDeviceFinishEvent, hv.layout(), hv.comp_node(), hv.raw_ptr(), dev_tensor().raw_ptr());
AsyncReleaser::inst()->add(hv);
}



+ 7
- 3
imperative/src/impl/profiler.cpp View File

@@ -37,11 +37,12 @@ std::shared_ptr<CompNode::Event> Timer::record_device(CompNode device) {
return event;
}

std::vector<Profiler::entry_t> Profiler::sm_records;
Profiler::options_t Profiler::sm_profile_options;
std::mutex Profiler::sm_mutex;
std::unordered_map<std::thread::id, Profiler*> Profiler::sm_profilers;
Timer Profiler::sm_timer;
profiler::HostTime Profiler::sm_start_at;
profiler::HostTime Profiler::sm_start_at = profiler::HostTime::min();
std::atomic_uint64_t Profiler::sm_last_id = 0;
bool Profiler::sm_profiling = false;
thread_local std::unique_ptr<Profiler> Profiler::tm_profiler = std::make_unique<Profiler>();
@@ -50,13 +51,16 @@ std::atomic_size_t Profiler::sm_preferred_capacity;
auto Profiler::get_thread_dict() -> thread_dict_t {
thread_dict_t thread_dict;
for (auto&& [tid, profiler]: sm_profilers) {
thread_dict[tid] = profiler->m_thread_name;
thread_dict[tid] = sys::get_thread_name(tid);
}
return thread_dict;
}

void Profiler::dump_profile(std::string basename, std::string format, bundle_t result) {
std::unordered_map<std::string, void(*)(std::string, bundle_t)> format_table;
static std::unordered_map<std::string, void(*)(std::string, bundle_t)> format_table = {
{"chrome_timeline.json", profiler::dump_chrome_timeline},
{"memory_flow.svg", profiler::dump_memory_flow},
};
auto iter = format_table.find(format);
if (iter == format_table.end()) {
mgb_log_error("unsupported profiling format %s", format.c_str());


+ 25
- 9
imperative/src/impl/profiler/chrome_timeline.cpp View File

@@ -86,10 +86,6 @@ public:
m_args[key] = value;
return *this;
}
ChromeTraceEvent& arg(std::string key, nlohmann::json value) {
m_args[key] = value;
return *this;
}
ChromeTraceEvent& stack(Trace trace) {
m_stack = std::move(trace);
return *this;
@@ -163,7 +159,7 @@ public:
return m_content.back();
}

std::string metadata(std::string key) {
std::string& metadata(std::string key) {
return m_metadata[key];
}

@@ -184,8 +180,8 @@ public:

std::string to_string() const {
auto json = to_json();
return "{" "traceEvents:" + nlohmann::to_string(json["traceEvents"]) + ","
"metadata:" + nlohmann::to_string(json["metadata"]) + "}";
return "{" "\"traceEvents\":" + nlohmann::to_string(json["traceEvents"]) + ","
"\"metadata\":" + nlohmann::to_string(json["metadata"]) + "}";
}
private:
std::vector<ChromeTraceEvent> m_content;
@@ -360,20 +356,40 @@ struct ChromeTimelineEventVisitor: EventVisitor<ChromeTimelineEventVisitor> {
new_host_event("AutoEvict", 'B');
} else if constexpr (std::is_same_v<TEvent, AutoEvictFinishEvent>) {
new_host_event("AutoEvict", 'E');
} else if constexpr (std::is_same_v<TEvent, HostToDeviceEvent>) {
new_device_event("HostToDevice", 'B', event.device);
} else if constexpr (std::is_same_v<TEvent, HostToDeviceFinishEvent>) {
new_device_event("HostToDevice", 'E', event.device)
.arg("shape", event.layout.TensorShape::to_string())
.arg("dtype", event.layout.dtype.name())
.arg("nr_elements", event.layout.total_nr_elems())
.arg("device", event.device.to_string());
}
}

void notify_counter(std::string key, int64_t old_val, int64_t new_val) {
new_host_event(key, 'C').arg("value", new_val);
}

void name_threads(Profiler::thread_dict_t thread_dict) {
for (auto&& [tid, tname]: thread_dict) {
trace_events.new_event()
.name("thread_name")
.pid('M')
.tid(to_tid(tid))
.arg("name", tname);
}
}
};


void dump_chrome_timeline(std::string filename, Profiler::bundle_t result){
ChromeTimelineEventVisitor visitor;
visitor.process_events(result);
visitor.trace_events.metadata("localTime") = std::to_string(result.start_at.time_since_epoch().count());
std::string json_repr = visitor.trace_events.to_string();
visitor.name_threads(result.thread_dict);
auto trace_events = std::move(visitor.trace_events);
trace_events.metadata("localTime") = std::to_string(result.start_at.time_since_epoch().count());
std::string json_repr = trace_events.to_string();
mgb::debug::write_to_file(filename.c_str(), json_repr);
}



+ 3
- 1
imperative/src/impl/profiler/states.h View File

@@ -234,7 +234,8 @@ public:
SampleDeviceEvent, WorkerExceptionEvent, ShapeInferEvent, SyncEvent, SyncFinishEvent,
StartProfileEvent, StartProfileFinishEvent, StopProfileEvent, StopProfileFinishEvent,
TensorCommandEvent, TensorCommandFinishEvent, AutoEvictEvent, AutoEvictFinishEvent,
CustomEvent, CustomFinishEvent, RecordDeviceEvent, ScopeEvent, ScopeFinishEvent> converter;
CustomEvent, CustomFinishEvent, RecordDeviceEvent, ScopeEvent, ScopeFinishEvent,
HostToDeviceEvent, HostToDeviceFinishEvent> converter;

auto for_each_entry = [&](auto&& handler) {
for (auto& entry: bundle.entries) {
@@ -308,6 +309,7 @@ public:
if constexpr (is_op_event<T>::value) {
current_op = &m_operators.at(event.op_id);
} else if constexpr (is_tensor_event<T>::value) {
mgb_assert(m_tensors.count(event.tensor_id) != 0, "tensor not found");
current_tensor = &m_tensors.at(event.tensor_id);
}
if constexpr (std::is_same_v<T, OpExecuteEvent>) {


+ 21
- 17
imperative/src/impl/profiler_plugin.cpp View File

@@ -31,6 +31,19 @@ ProfilerPlugin::ProfilerPlugin(cg::ComputingGraph* graph): PluginBase(graph) {
if (m_opr_dict.empty() && m_var_dict.empty()) {
init_seq(event.exec);
}
Profiler::record<ScopeEvent>("Constants");
for (auto&& [var, var_info]: m_var_dict) {
if (var_info->is_const) {
bool valid = var->dev_tensor_valid();
auto layout = valid ? var->layout() : TensorLayout();
var_info->id = Profiler::next_id();
Profiler::record<TensorDeclareEvent>(var_info->id, var->name());
Profiler::record<TensorProduceEvent>(var_info->id, layout, var->comp_node(), valid ? var->dev_tensor().raw_ptr() : nullptr);
} else {
var_info->rt_ref_cnt = var_info->ref_cnt;
}
}
Profiler::record<ScopeFinishEvent>("Constants");
Profiler::record<ScopeEvent>("DispatchOprs");
event.exec->iter_opr_seq([this](OperatorNodeBase* opr) -> bool{
auto& opr_info = get_opr_info(opr);
@@ -44,26 +57,15 @@ ProfilerPlugin::ProfilerPlugin(cg::ComputingGraph* graph): PluginBase(graph) {
}
auto opr_name = opr->dyn_typeinfo()->name;
auto copy_params = [params = opr_info.params] { return *params; };
Profiler::record<OpDispatchEvent>(opr_info.id, opr_name, copy_params, inputs, outputs);
for (auto output: opr->output()) {
auto var_id = get_var_info(output).id;
Profiler::record<TensorDeclareEvent>(var_id);
auto& var_id = get_var_info(output).id;
var_id = Profiler::next_id();
Profiler::record<TensorDeclareEvent>(var_id, output->name());
}
Profiler::record<OpDispatchEvent>(opr_info.id, opr_name, copy_params, inputs, outputs);
return true;
});
Profiler::record<ScopeFinishEvent>("DispatchOprs");
Profiler::record<ScopeEvent>("Constants");
for (auto&& [var, var_info]: m_var_dict) {
if (var_info->is_const) {
bool valid = var->dev_tensor_valid();
auto layout = valid ? var->layout() : TensorLayout();
Profiler::record<TensorDeclareEvent>(var_info->id);
Profiler::record<TensorProduceEvent>(var_info->id, layout, var->comp_node(), valid ? var->dev_tensor().raw_ptr() : nullptr);
} else {
var_info->rt_ref_cnt = var_info->ref_cnt;
}
}
Profiler::record<ScopeFinishEvent>("Constants");
};
auto on_opr_start = [this](OprExecStart const& event) {
OperatorNodeBase* opr = event.opr;
@@ -144,6 +146,7 @@ ProfilerPlugin::ProfilerPlugin(cg::ComputingGraph* graph): PluginBase(graph) {
Profiler::record<TensorReleaseEvent>(var_info->id);
}
Profiler::record<TensorEraseEvent>(var_info->id, var_info->ref_cnt);
var_info->id = 0;
}
};
add_event_handler(graph->event().register_receiver<CompSeqExecBeforeStart>(on_seq_start));
@@ -194,11 +197,12 @@ ProfilerPlugin::OprInfo& ProfilerPlugin::register_opr(cg::OperatorNodeBase *opr)

ProfilerPlugin::VarInfo& ProfilerPlugin::register_var(cg::VarNode *var) {
auto info = std::make_unique<VarInfo>();
info->id = Profiler::next_id();
info->id = 0;
info->is_const = false;
info->ref_cnt = 0;
info->rt_ref_cnt = 0;
return *m_var_dict.insert({var, std::move(info)}).first->second;
mgb_assert(m_var_dict.count(var) == 0, "var exists");
return *(m_var_dict[var] = std::move(info));
}

ProfilerPlugin::OprInfo& ProfilerPlugin::get_opr_info(cg::OperatorNodeBase *opr) {


+ 9
- 14
imperative/src/include/megbrain/imperative/profiler.h View File

@@ -81,10 +81,9 @@ public:
private:
std::thread::id m_thread_id;
std::vector<Record> m_records;
std::vector<std::any> m_duration_stack;
std::atomic<Status> m_status = Running;
std::string m_thread_name;

static std::vector<entry_t> sm_records;
static options_t sm_profile_options;
static std::mutex sm_mutex;
static std::unordered_map<std::thread::id, Profiler*> sm_profilers;
@@ -99,9 +98,6 @@ public:
Profiler() {
m_thread_id = std::this_thread::get_id();
MGB_LOCK_GUARD(sm_mutex);
if (sm_profilers.size() == 0) {
reset();
}
mgb_assert(sm_profilers.count(m_thread_id) == 0);
sm_profilers[m_thread_id] = this;
}
@@ -109,17 +105,13 @@ public:
MGB_LOCK_GUARD(sm_mutex);
mgb_assert(sm_profilers.count(m_thread_id) == 1);
sm_profilers.erase(m_thread_id);
sm_records.insert(sm_records.end(), m_records.begin(), m_records.end());
}
public:
static Profiler& get_instance() {
return *tm_profiler;
}

static void reset() {
mgb_assert(sm_profilers.size() == 0, "profiler already running");
sm_start_at = profiler::HostTime::min();
}

static uint64_t next_id() {
return sm_last_id++;
}
@@ -151,12 +143,10 @@ public:
mgb_assert(profiler->m_status.compare_exchange_strong(expected, Collecting));
}
}
std::vector<entry_t> profile_data;
std::vector<entry_t> profile_data = std::move(sm_records);
for (auto&& [tid, profiler]: sm_profilers) {
sm_preferred_capacity = std::max(sm_preferred_capacity.load(), profiler->m_records.size());
for (auto& record: profiler->m_records) {
profile_data.push_back(std::move(record));
}
profile_data.insert(profile_data.end(), profiler->m_records.begin(), profiler->m_records.end());
profiler->m_records.clear();
profiler->m_records.reserve(sm_preferred_capacity);
}
@@ -238,5 +228,10 @@ public:
}
};

#define MGB_RECORD_EVENT(type, ...) \
if (mgb::imperative::Profiler::is_profiling()) { \
mgb::imperative::Profiler::record<type>(type{__VA_ARGS__}); \
} \

} // namespace imperative
} // namespace mgb

Loading…
Cancel
Save