!1350 buffer_pool_allocator

From: @selfws
Reviewed-by: @xchu42
Signed-off-by:
pull/1350/MERGE
mindspore-ci-bot 4 years ago committed by Gitee
commit c9a4b65720

@ -329,6 +329,7 @@ set(TRAIN_SRC_LIST
"graph/passes/memcpy_addr_async_pass.cc"
"graph/passes/parallel_group_pass.cc"
"graph/passes/set_input_output_offset_pass.cc"
"graph/passes/buffer_pool_memory_pass.cc"
"graph/preprocess/graph_preprocess.cc"
"graph/preprocess/insert_op/ge_aipp_op.cc"
"graph/preprocess/insert_op/util_insert_aipp_op.cc"
@ -407,6 +408,7 @@ set(TRAIN_SRC_LIST
"graph/build/memory/hybrid_mem_assigner.cc"
"graph/build/memory/max_block_mem_assigner.cc"
"graph/build/memory/var_mem_assign_util.cc"
"graph/build/memory/buffer_pool_mem_assigner.cc"
)
set(INFER_SRC_LIST
@ -617,6 +619,7 @@ set(INFER_SRC_LIST
"graph/passes/memcpy_addr_async_pass.cc"
"graph/passes/set_input_output_offset_pass.cc"
"graph/passes/parallel_group_pass.cc"
"graph/passes/buffer_pool_memory_pass.cc"
"graph/manager/model_manager/event_manager.cc"
"graph/manager/util/rt_context_util.cc"
"graph/manager/util/variable_accelerate_ctrl.cc"
@ -680,6 +683,7 @@ set(INFER_SRC_LIST
"graph/build/memory/hybrid_mem_assigner.cc"
"graph/build/memory/max_block_mem_assigner.cc"
"graph/build/memory/var_mem_assign_util.cc"
"graph/build/memory/buffer_pool_mem_assigner.cc"
)
if (NOT ENABLE_D AND NOT ENABLE_ACL AND NOT ENABLE_MS_TESTCASES)

@ -222,6 +222,7 @@ OMG_HOST_SRC_FILES := \
graph/passes/hccl_group_pass.cc \
graph/passes/memcpy_addr_async_pass.cc \
graph/passes/set_input_output_offset_pass.cc \
graph/passes/buffer_pool_memory_pass.cc \
OMG_DEVICE_SRC_FILES := $(OMG_HOST_SRC_FILES)

@ -246,6 +246,7 @@ LIBGE_LOCAL_SRC_FILES := \
graph/passes/end_of_sequence_add_control_pass.cc \
graph/passes/memcpy_addr_async_pass.cc \
graph/passes/set_input_output_offset_pass.cc \
graph/passes/buffer_pool_memory_pass.cc \
graph/preprocess/graph_preprocess.cc \
graph/preprocess/insert_op/ge_aipp_op.cc \
graph/preprocess/insert_op/util_insert_aipp_op.cc \

@ -1655,6 +1655,8 @@ Status BlockMemAssigner::AssignOutputMemoryWithReuse(const NodePtr &node, vector
bool is_atomic = false;
// If GetBool fail, is_atomic is false.
(void)ge::AttrUtils::GetBool(op_desc, ATOMIC_ATTR_IS_ATOMIC_NODE, is_atomic);
bool is_buffer_pool_mem_supported = (op_desc->HasAttr(ATTR_NAME_BUFFER_POOL_ID)) &&
(op_desc->HasAttr(ATTR_NAME_BUFFER_POOL_SIZE)) && (!root_unknown_shape_flag_);
// Allocate memory for the current node and release node memory of the same size in the workspace
GE_IF_BOOL_EXEC(ge_disable_reuse_mem_env_ != "1",
for (auto iter = stream_workspace_blocks_.begin(); iter != stream_workspace_blocks_.end();
@ -1694,7 +1696,7 @@ Status BlockMemAssigner::AssignOutputMemoryWithReuse(const NodePtr &node, vector
GE_IF_BOOL_EXEC(!no_need_assign_memory,
no_need_assign_memory = IsAtomicOutputMemory(node, i, is_atomic, out_node_set_continuous_input););
}
no_need_assign_memory = (no_need_assign_memory || IsKnownSubgraphData(node));
no_need_assign_memory = (no_need_assign_memory || IsKnownSubgraphData(node) || is_buffer_pool_mem_supported);
if (no_need_assign_memory) {
zero_memory_list_.emplace_back(node, kOutput, i, false);
continue;
@ -1740,6 +1742,13 @@ void BlockMemAssigner::AssignMemoryWithReuse(vector<int64_t> &ranges) {
const char *op_no_reuse_mem = std::getenv(OP_NO_REUSE_MEM);
GE_IF_BOOL_EXEC(op_no_reuse_mem != nullptr, op_no_reuse_mem_str = string(op_no_reuse_mem);
CheckAndGetOpReuseEnv(op_no_reuse_mem_str, op_no_reuse_mem_vec_, op_reuse_env_valid_););
auto root_graph = GraphUtils::FindRootGraph(compute_graph_);
if (root_graph == nullptr) {
GELOGE(INTERNAL_ERROR, "[Check][RootGraph]Root graph is nullptr, graph:%s.", compute_graph_->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Root graph is nullptr, graph:%s.", compute_graph_->GetName().c_str());
return;
}
root_unknown_shape_flag_ = root_graph->GetGraphUnknownFlag();
for (NodePtr &n : compute_graph_->GetAllNodes()) {
auto node_op_desc = n->GetOpDesc();

@ -494,6 +494,8 @@ class BlockMemAssigner : public MemAssigner {
/// @ [stream2][nodeid]
///
DependStreamLife total_node_depend_stream_life_;
bool root_unknown_shape_flag_ = false;
};
} // namespace ge
#endif // GE_GRAPH_BUILD_MEMORY_BLOCK_MEM_ASSIGNER_H_

@ -0,0 +1,234 @@
/**
* Copyright 2020 Huawei Technologies Co., Ltd
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "graph/build/memory/buffer_pool_mem_assigner.h"
#include "graph/common/omg_util.h"
#include "graph/utils/tensor_utils.h"
#include "framework/common/util.h"
#include "graph/compute_graph.h"
#include "graph/debug/ge_attr_define.h"
#include "common/math/math_util.h"
#include "common/util/error_manager/error_manager.h"
namespace ge {
namespace {
const size_t kBufferPoolNodeMemInfoLength = 2;
const uint32_t kBufferPoolNodeOutputSizeIndex = 0;
const uint32_t kBufferPoolNodeOutputOffsetIndex = 1;
} // namespace
Status BufferPoolMemAssigner::Assign() {
if (compute_graph_ == nullptr) {
GELOGE(PARAM_INVALID, "[Check][Graph]Graph is nullptr");
REPORT_INNER_ERROR("E19999", "Input graph is nullptr");
return PARAM_INVALID;
}
Status ret = InitAssigner(compute_graph_);
if (ret != SUCCESS) {
GELOGE(FAILED, "[Init][Assigner]Graph:%s.", compute_graph_->GetName().c_str());
return FAILED;
}
ret = AssignOutput();
if (ret != SUCCESS) {
GELOGE(FAILED, "[Assign][Output]Graph:%s.", compute_graph_->GetName().c_str());
return FAILED;
}
return SUCCESS;
}
Status BufferPoolMemAssigner::GetOutputMemoryType(const NodePtr &node, size_t idx, int64_t &memory_type) {
GE_CHECK_NOTNULL(node->GetOpDesc());
memory_type = RT_MEMORY_HBM;
std::vector<int64_t> type_list;
bool has_mem_type = ge::AttrUtils::GetListInt(node->GetOpDesc(), ATTR_NAME_OUTPUT_MEM_TYPE_LIST, type_list);
if (has_mem_type && (type_list.size() != node->GetOpDesc()->GetOutputsSize() || idx >= type_list.size())) {
GELOGE(PARAM_INVALID, "[Check][OutputParam]Output param invalid, output size:%zu, mem type size:%zu, index:%zu.",
node->GetOpDesc()->GetOutputsSize(), type_list.size(), idx);
REPORT_INNER_ERROR("E19999", "Output param invalid, output size:%zu, mem type size:%zu, index:%zu.",
node->GetOpDesc()->GetOutputsSize(), type_list.size(), idx);
return PARAM_INVALID;
}
memory_type = has_mem_type ? type_list[idx] : RT_MEMORY_HBM;
return SUCCESS;
}
Status BufferPoolMemAssigner::InitAssigner(const ComputeGraphPtr &graph) {
for (const NodePtr &node : graph->GetAllNodes()) {
int64_t buffer_pool_id = 0;
int64_t buffer_pool_size = 0;
bool get_attr = AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_BUFFER_POOL_ID, buffer_pool_id);
get_attr = get_attr && (AttrUtils::GetInt(node->GetOpDesc(), ATTR_NAME_BUFFER_POOL_SIZE, buffer_pool_size));
if (get_attr) {
std::string batch_label;
(void) AttrUtils::GetStr(node->GetOpDesc(), ATTR_NAME_BATCH_LABEL, batch_label);
buffer_pool_nodes_[batch_label][buffer_pool_id].emplace_back(node);
auto iter = buffer_pool_size_[batch_label].find(buffer_pool_id);
if (iter == buffer_pool_size_[batch_label].end()) {
buffer_pool_size_[batch_label][buffer_pool_id] = buffer_pool_size;
}
Status ret = InitMemOffsetBase(node);
if (ret != SUCCESS) {
GELOGE(ret, "[Init][MemOffsetBase]Batch label:%s.", batch_label.c_str());
REPORT_INNER_ERROR("E19999", "Failed to init offset base, batch label:%s.", batch_label.c_str());
return ret;
}
}
}
int64_t max_size = 0;
for (const auto &iter : buffer_pool_size_) {
std::string batch_label = iter.first;
int64_t batch_offset = mem_offset_base_;
for (const auto &buffer_pool : iter.second) {
int64_t buffer_pool_id = buffer_pool.first;
int64_t buffer_pool_size = buffer_pool.second;
buffer_pool_offset_base_[batch_label][buffer_pool_id] = batch_offset;
FMK_INT64_ADDCHECK(buffer_pool_size, kBufferPoolMemAlignSize);
AlignMemSize(buffer_pool_size, kBufferPoolMemAlignSize);
FMK_INT64_ADDCHECK(batch_offset, (buffer_pool_size + kBufferPoolMemAlignSize));
batch_offset += (buffer_pool_size + kBufferPoolMemAlignSize);
}
int64_t batch_mem_size = batch_offset - mem_offset_base_;
GELOGI("[Init][Assigner]Get batch mem size, batch label:%s, mem size:%ld.", batch_label.c_str(), batch_mem_size);
if (max_size < batch_mem_size) {
max_size = batch_mem_size;
}
}
FMK_INT64_ADDCHECK(mem_offset_base_, max_size);
mem_offset_ = static_cast<size_t>(mem_offset_base_ + max_size);
GELOGI("[Init][Assigner]Init buffer pool mem assigner successfully, "
"mem type:%ld, mem offset base:%ld, mem offset:%zu.", mem_type_, mem_offset_base_, mem_offset_);
return SUCCESS;
}
Status BufferPoolMemAssigner::InitMemOffsetBase(const NodePtr &node) {
int64_t mem_type;
Status ret = GetOutputMemoryType(node, static_cast<size_t>(kBufferPoolNodeOutIndex), mem_type);
if (ret != SUCCESS) {
GELOGE(ret, "[Get][MemType]Node:%s, index:%u.", node->GetName().c_str(), kBufferPoolNodeOutIndex);
REPORT_INNER_ERROR("E19999", "Failed to get output memory type, node:%s, index:%u.",
node->GetName().c_str(), kBufferPoolNodeOutIndex);
return ret;
}
if (mem_type_ != mem_type && init_offset_base_) {
GELOGE(PARAM_INVALID, "[Check][MemType]The memory type of all buffer pool nodes must be the same, node:%s, "
"required:%ld, actually: %ld", node->GetName().c_str(), mem_type_, mem_type);
REPORT_INNER_ERROR("E19999", "The memory type of all buffer pool nodes must be the same, node:%s, "
"required:%ld, actually: %ld", node->GetName().c_str(), mem_type_, mem_type);
return PARAM_INVALID;
}
if (!init_offset_base_) {
auto iter = mem_type_to_offset_.find(mem_type);
if (iter == mem_type_to_offset_.end()) {
GELOGE(PARAM_INVALID, "[Check][MemType]Memory type is not supported, node:%s, mem type:%ld.",
node->GetName().c_str(), mem_type);
REPORT_INNER_ERROR("E19999", "Memory type is not supported, node:%s, mem type:%ld.",
node->GetName().c_str(), mem_type);
return PARAM_INVALID;
}
mem_offset_base_ = static_cast<int64_t>(iter->second);
FMK_INT64_ADDCHECK(mem_offset_base_, (kBufferPoolMemAlignSize + kBufferPoolMemAlignSize));
AlignMemSize(mem_offset_base_, kBufferPoolMemAlignSize);
// The HCOM nodes may access the previous 512 bytes.
mem_offset_base_ += kBufferPoolMemAlignSize;
mem_type_ = mem_type;
init_offset_base_ = true;
GELOGI("[Init][MemOffsetBase]Init offset base:%ld, memory type:%ld", mem_offset_base_, mem_type);
}
return SUCCESS;
}
Status BufferPoolMemAssigner::AssignOutput() {
for (auto &batch_pool_nodes_map : buffer_pool_nodes_) {
std::string batch_label = batch_pool_nodes_map.first;
for (auto &pool_nodes_map : batch_pool_nodes_map.second) {
int64_t buffer_pool_id = pool_nodes_map.first;
auto iter_buffer_id_size = buffer_pool_size_[batch_label].find(buffer_pool_id);
if (iter_buffer_id_size == buffer_pool_size_[batch_label].end()) {
GELOGE(INTERNAL_ERROR, "[Get][BufferPoolSize]Pool id:%ld.", buffer_pool_id);
REPORT_INNER_ERROR("E19999", "Failed to get buffer pool size, pool id:%ld.", buffer_pool_id);
return INTERNAL_ERROR;
}
auto iter_buffer_id_offset = buffer_pool_offset_base_[batch_label].find(buffer_pool_id);
if (iter_buffer_id_offset == buffer_pool_offset_base_[batch_label].end()) {
GELOGE(INTERNAL_ERROR, "[Get][BufferPoolBaseOffset]Pool id:%ld.", buffer_pool_id);
REPORT_INNER_ERROR("E19999", "Failed to get buffer pool base offset, pool id:%ld.", buffer_pool_id);
return INTERNAL_ERROR;
}
int64_t buffer_pool_size = iter_buffer_id_size->second;
int64_t output_offset_base = iter_buffer_id_offset->second;
Status ret = AssignOutputInOneBufferPool(batch_label, output_offset_base, pool_nodes_map.second);
if (ret != SUCCESS) {
GELOGE(ret, "[Assign][OneBufferPool]Batch label:%s, pool id:%ld, pool size:%ld, offset base:%ld.",
batch_label.c_str(), buffer_pool_id, buffer_pool_size, output_offset_base);
REPORT_INNER_ERROR("E19999", "Failed to assign output memory, batch label:%s, "
"pool id:%ld, pool size:%ld, offset base:%ld.",
batch_label.c_str(), buffer_pool_id, buffer_pool_size, output_offset_base);
return ret;
}
GELOGI("[Assign][Output]Assign output successfully, batch label:%s, pool id:%ld, pool size:%ld, offset base:%ld.",
batch_label.c_str(), buffer_pool_id, buffer_pool_size, output_offset_base);
}
}
return SUCCESS;
}
Status BufferPoolMemAssigner::AssignOutputInOneBufferPool(const std::string &batch_label,
int64_t output_offset_base,
const std::vector<NodePtr> &buffer_pool_nodes) {
for (const NodePtr &node : buffer_pool_nodes) {
int64_t output_size = 0;
Status ret = GetMemorySize(node, output_size);
if (ret != SUCCESS) {
GELOGE(ret, "[Get][MemSize]Node:%s.", node->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Failed to get output size, node:%s.", node->GetName().c_str());
return ret;
}
OpDescPtr op_desc = node->GetOpDesc();
GE_CHECK_NOTNULL(op_desc);
vector<int64_t> memory_size_and_offset;
bool get_attr = AttrUtils::GetListInt(op_desc, ATTR_NAME_BUFFER_POOL_NODE_SIZE_AND_OFFSET, memory_size_and_offset);
if (!get_attr || memory_size_and_offset.size() != kBufferPoolNodeMemInfoLength) {
GELOGE(PARAM_INVALID, "[Get][Attr]Node:%s, mem info size:%zu, required size:%zu.",
node->GetName().c_str(), memory_size_and_offset.size(), kBufferPoolNodeMemInfoLength);
REPORT_INNER_ERROR("E19999", "Failed to get pool node memory info, node:%s, info size:%zu, required size:%zu.",
node->GetName().c_str(), memory_size_and_offset.size(), kBufferPoolNodeMemInfoLength);
return PARAM_INVALID;
}
if (output_size != memory_size_and_offset[kBufferPoolNodeOutputSizeIndex]) {
GELOGE(PARAM_INVALID, "[Check][MemSize]Something wrong with memory size, pre size:%ld, curr size:%ld, node:%s.",
memory_size_and_offset[kBufferPoolNodeOutputSizeIndex], output_size, node->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Something wrong with memory size, pre size:%ld, curr size:%ld, node:%s.",
memory_size_and_offset[kBufferPoolNodeOutputSizeIndex], output_size, node->GetName().c_str());
return PARAM_INVALID;
}
int64_t logical_offset = memory_size_and_offset[kBufferPoolNodeOutputOffsetIndex];
vector<int64_t> output_list = {(output_offset_base + logical_offset)};
op_desc->SetOutputOffset(output_list);
// log for IMAS tools
GELOGI("[IMAS]Set %s name[%s] optype[%s] %s[%u] offset to [%ld] streamid[%ld] memtype[%ld] "
"size[%zu] realsize[%zu] noalignsize[%zu] life time begin[%d] life time end[%d] "
"child[%d:%d:%d:%d:%d] isref[%d] batch[%s]",
compute_graph_->GetName().c_str(), op_desc->GetName().c_str(), op_desc->GetType().c_str(),
"output", kBufferPoolNodeOutIndex, output_list[kBufferPoolNodeOutIndex], op_desc->GetStreamId(), mem_type_,
static_cast<size_t>(output_size), static_cast<size_t>(output_size), static_cast<size_t>(output_size),
0, 0, 0, 0, 0, 0, 0, 0, batch_label.c_str());
}
return SUCCESS;
}
} // namespace ge

@ -0,0 +1,83 @@
/**
* Copyright 2020 Huawei Technologies Co., Ltd
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef GE_GRAPH_BUILD_MEMORY_BUFFER_POOL_MEM_ASSIGNER_H_
#define GE_GRAPH_BUILD_MEMORY_BUFFER_POOL_MEM_ASSIGNER_H_
#include <vector>
#include <map>
#include <unordered_map>
#include "graph/build/memory/mem_assigner.h"
#include "runtime/mem.h"
namespace ge {
class BufferPoolMemAssigner : public MemAssigner {
public:
BufferPoolMemAssigner(ComputeGraphPtr compute_graph, const std::map<int64_t, size_t> &mem_type_to_offset)
: MemAssigner(), compute_graph_(compute_graph),
mem_type_(0),
mem_offset_(0),
mem_offset_base_(0),
init_offset_base_(false),
mem_type_to_offset_(mem_type_to_offset) {}
BufferPoolMemAssigner(const BufferPoolMemAssigner &) = delete;
BufferPoolMemAssigner &operator=(const BufferPoolMemAssigner &) = delete;
~BufferPoolMemAssigner() override = default;
Status Assign() override;
size_t GetMemOffset() const { return mem_offset_; }
int64_t GetMemType() const { return mem_type_; }
private:
static Status GetOutputMemoryType(const NodePtr &node, size_t idx, int64_t &memory_type);
Status InitAssigner(const ComputeGraphPtr &graph);
Status InitMemOffsetBase(const NodePtr &node);
Status AssignOutput();
Status AssignOutputInOneBufferPool(const std::string &batch_label,
int64_t output_offset_base,
const std::vector<NodePtr> &buffer_pool_nodes);
ComputeGraphPtr compute_graph_;
int64_t mem_type_;
size_t mem_offset_;
int64_t mem_offset_base_;
bool init_offset_base_;
std::map<int64_t, size_t> mem_type_to_offset_;
// Use map to ensure that each visit is in the order of pool id
std::unordered_map<std::string, std::map<int64_t, std::vector<NodePtr>>> buffer_pool_nodes_;
// Use map to ensure that each visit is in the order of pool id
std::unordered_map<std::string, std::map<int64_t, int64_t>> buffer_pool_size_;
std::unordered_map<std::string, std::unordered_map<int64_t, int64_t>> buffer_pool_offset_base_;
};
} // namespace ge
#endif // GE_GRAPH_BUILD_MEMORY_BUFFER_POOL_MEM_ASSIGNER_H_

@ -30,6 +30,7 @@
#include "graph/manager/graph_var_manager.h"
#include "graph/utils/tensor_utils.h"
#include "graph/utils/type_utils.h"
#include "graph/build/memory/buffer_pool_mem_assigner.h"
namespace {
const int kAllInputAddrIsAtomic = -1;
@ -231,6 +232,7 @@ Status GraphMemoryAssigner::ReAssignMemory(bool is_loop_graph, map<int64_t, size
GE_CHK_STATUS_RET(ReAssignContinuousMemory(is_loop_graph), "ReAssignContinuousMemory Failed!");
GE_CHK_STATUS_RET(ReAssignAtomicMemory(is_loop_graph), "ReAssignAtomicMemory Failed!");
GE_CHK_STATUS_RET(AssignBufferPoolMemory(), "AssignBufferPoolMemory Failed!");
size_t total_mem_offset = 0;
for (auto pair : memory_offset_) {
@ -1735,4 +1737,54 @@ ge::Status GraphMemoryAssigner::AssignContinuousInputMemoryWithAtomicProcess(con
return ge::SUCCESS;
}
Status GraphMemoryAssigner::AssignBufferPoolMemory() {
auto is_buffer_pool_mem_enable = [] (const ComputeGraphPtr &graph) -> bool {
for (NodePtr &node : graph->GetAllNodes()) {
auto op_desc = node->GetOpDesc();
if (op_desc == nullptr) {
continue;
}
bool has_attrs = op_desc->HasAttr(ATTR_NAME_BUFFER_POOL_ID) && op_desc->HasAttr(ATTR_NAME_BUFFER_POOL_SIZE);
if (has_attrs) {
return true;
}
}
return false;
};
auto root_graph = GraphUtils::FindRootGraph(compute_graph_);
GE_CHECK_NOTNULL(root_graph);
if (root_graph->GetGraphUnknownFlag()) {
GELOGI("[Check][Enable]Unknown root graph does not support buffer pool memory, graph:%s.",
compute_graph_->GetName().c_str());
return SUCCESS;
}
if (!is_buffer_pool_mem_enable(compute_graph_)) {
GELOGD("[Check][Enable]Buffer pool memory is not enable, graph:%s.", compute_graph_->GetName().c_str());
return SUCCESS;
}
map<int64_t, size_t> mem_type_to_offset;
for (const auto &pair : memory_offset_) {
mem_type_to_offset[pair.first] = pair.second.mem_offset_;
}
BufferPoolMemAssigner buffer_pool_mem_assigner(compute_graph_, mem_type_to_offset);
Status status = buffer_pool_mem_assigner.Assign();
if (status != SUCCESS) {
GELOGE(status, "[Assign][BufferPoolMem]Graph:%s.", compute_graph_->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Failed to assign buffer pool memory, graph:%s.", compute_graph_->GetName().c_str());
return status;
}
int64_t mem_type = buffer_pool_mem_assigner.GetMemType();
auto iter = memory_offset_.find(mem_type);
if (iter == memory_offset_.end()) {
GELOGE(FAILED, "[Check][MemType]Memory type is not supported, graph:%s, mem type:%ld.",
compute_graph_->GetName().c_str(), mem_type);
REPORT_INNER_ERROR("E19999", "Memory type is not supported, graph:%s, mem type:%ld.",
compute_graph_->GetName().c_str(), mem_type);
return FAILED;
}
iter->second.mem_offset_ = buffer_pool_mem_assigner.GetMemOffset();
GELOGI("[Assign][BufferPoolMem]Assign buffer pool memory successfully, graph:%s, mem type:%ld, mem offset:%zu.",
compute_graph_->GetName().c_str(), mem_type, buffer_pool_mem_assigner.GetMemOffset());
return SUCCESS;
}
} // namespace ge

@ -188,6 +188,8 @@ class GraphMemoryAssigner {
void PrintMemoryOffset();
Status AssignBufferPoolMemory();
MemoryOffsetMap memory_offset_;
ge::ComputeGraphPtr compute_graph_;
HybridMemAssignerPtr mem_assigner_;

@ -8,6 +8,7 @@ local_lib_src_files := memory_assigner.cc \
hybrid_mem_assigner.cc \
max_block_mem_assigner.cc \
var_mem_assign_util.cc \
buffer_pool_mem_assigner.cc \
local_lib_inc_path := ${LOCAL_PATH} \
${TOPDIR}inc \

@ -18,6 +18,7 @@
#include "common/util.h"
#include "framework/common/debug/ge_log.h"
#include "graph/debug/ge_attr_define.h"
#include "graph/common/omg_util.h"
namespace ge {
RunContextUtil::~RunContextUtil() { DestroyRtModelResources(); }
@ -88,9 +89,11 @@ Status RunContextUtil::CreateRtModelResources(uint32_t stream_num, uint32_t even
}
// Create rt event
uint32_t create_flag = static_cast<uint32_t>((event_num > kEventReuseThreshold) ? RT_EVENT_WITH_FLAG :
RT_EVENT_DEFAULT);
for (uint32_t i = 0; i < event_num; ++i) {
rtEvent_t event = nullptr;
rt_ret = rtEventCreate(&event);
rt_ret = rtEventCreateWithFlag(&event, create_flag);
if (rt_ret != RT_ERROR_NONE) {
REPORT_CALL_ERROR("E19999", "call rtEventCreate fail, ret:%d, index:%u, when %s",
static_cast<int>(rt_ret), i, __FUNCTION__);

File diff suppressed because it is too large Load Diff

@ -71,6 +71,10 @@ class StreamAllocator {
Status SetActiveStreamsForLoop();
Status CheckStreamActived() const;
Status ReuseEvent(bool send_to,
const std::unordered_map<std::string, ge::NodePtr> &name_to_node_map,
const std::unordered_map<ge::NodePtr, std::vector<std::pair<std::string, uint32_t>>> &node_to_event_id);
Status RefreshEventsWithReuse();
Status RefreshContinuousEvents();
Status InsertSyncEventNodes();

@ -21,6 +21,8 @@
#include "framework/common/debug/ge_log.h"
#include "graph/debug/ge_attr_define.h"
#include "graph/utils/graph_utils.h"
#include "graph/utils/tensor_utils.h"
#include "common/math/math_util.h"
namespace ge {
///
@ -204,4 +206,42 @@ Status SetNextIteration(const ge::NodePtr &node, const std::string &next) {
return SUCCESS;
}
///
/// @brief Align the memory
/// @param [in/out] memory size
/// @param [in] alinment
/// @return void
///
void AlignMemSize(int64_t &mem_size, int64_t align_size) {
if (mem_size <= 0) {
return;
}
mem_size = (mem_size + align_size - 1) / align_size * align_size;
}
///
/// @brief Get memory size from tensor desc
/// @param [in] node
/// @param [out] memory size
/// @return Status
///
Status GetMemorySize(const NodePtr &node, int64_t &output_size) {
GE_CHECK_NOTNULL(node->GetOpDesc());
auto output_op_desc = node->GetOpDesc()->GetOutputDescPtr(kBufferPoolNodeOutIndex);
GE_CHECK_NOTNULL(output_op_desc);
int64_t size = 0;
auto ret = ge::TensorUtils::GetSize(*output_op_desc, size);
if (ret != ge::GRAPH_SUCCESS) {
GELOGE(INTERNAL_ERROR, "[Get][Size]Node:%s.", node->GetName().c_str());
REPORT_INNER_ERROR("E19999", "Failed to get output size, node:%s.", node->GetName().c_str());
return INTERNAL_ERROR;
}
FMK_INT64_ADDCHECK(size, kBufferPoolMemAlignSize);
AlignMemSize(size, kBufferPoolMemAlignSize);
// The HCOM operator requires an additional 512 bytes before and after
FMK_INT64_ADDCHECK(size, (kBufferPoolMemAlignSize + kBufferPoolMemAlignSize));
output_size = kBufferPoolMemAlignSize + size + kBufferPoolMemAlignSize;
return SUCCESS;
}
} // namespace ge

@ -27,6 +27,11 @@
#include "graph/node.h"
namespace ge {
namespace {
const int64_t kBufferPoolMemAlignSize = 512;
const uint32_t kBufferPoolNodeOutIndex = 0;
const uint32_t kEventReuseThreshold = 65500;
} // namespace
///
/// @brief get the Original Type of FrameworkOp
/// @param [in] node
@ -96,6 +101,22 @@ Status SetCyclicDependenceFlag(const ge::NodePtr &node);
/// @return Status
///
Status SetNextIteration(const ge::NodePtr &node, const std::string &next);
///
/// @brief Align the memory
/// @param [in/out] memory size
/// @param [in] alinment
/// @return void
///
void AlignMemSize(int64_t &mem_size, int64_t align_size);
///
/// @brief Get memory size from tensor desc
/// @param [in] node
/// @param [out] memory size
/// @return Status
///
Status GetMemorySize(const NodePtr &node, int64_t &output_size);
} // namespace ge
#endif // GE_GRAPH_COMMON_OMG_UTIL_H_

@ -60,6 +60,7 @@
#include "securec.h"
#include "graph/common/local_context.h"
#include "common/formats/utils/formats_trans_utils.h"
#include "graph/common/omg_util.h"
// create std::thread, catch exceptions using try/catch
#define CREATE_STD_THREAD(thread_id, func, args) \
@ -664,9 +665,12 @@ Status DavinciModel::Init(void *dev_ptr, size_t mem_size, void *weight_ptr, size
GELOGI("Logical stream index:%u, stream:%p, rtstream: %d.", i, stream, rt_stream_id);
}
for (uint32_t i = 0; i < EventNum(); i++) {
rtEvent_t rt_event;
GE_CHK_RT_RET(rtEventCreate(&rt_event));
uint32_t event_num = EventNum();
uint32_t create_flag = static_cast<uint32_t>((event_num > kEventReuseThreshold) ? RT_EVENT_WITH_FLAG :
RT_EVENT_DEFAULT);
for (uint32_t i = 0; i < event_num; ++i) {
rtEvent_t rt_event = nullptr;
GE_CHK_RT_RET(rtEventCreateWithFlag(&rt_event, create_flag));
event_list_.push_back(rt_event);
}

@ -95,6 +95,7 @@
#include "graph/passes/memcpy_addr_async_pass.h"
#include "graph/passes/hccl_continuous_memcpy_pass.h"
#include "graph/passes/parallel_group_pass.h"
#include "graph/passes/buffer_pool_memory_pass.h"
#include "graph/build/label_allocator.h"
#include "graph/utils/tensor_adapter.h"
#include "inc/pass_manager.h"
@ -2528,6 +2529,12 @@ Status GraphManager::OptimizeStage2(ge::ComputeGraphPtr &compute_graph) {
GE_CHK_STATUS_RET(memcpy_addr.Run(compute_graph), "Add memcpy_addr_async node failed.");
GE_TIMESTAMP_END(AddMemcpyAddrAsyncNode, "MemcpyAddrAsyncPass::Run.");
// Process offset and dependency for buffer pool memory assigner.
GE_TIMESTAMP_START(BufferPoolMemoryPass);
BufferPoolMemoryPass buffer_pool_mem_pass;
GE_CHK_STATUS_RET(buffer_pool_mem_pass.Run(compute_graph), "Failed to process for buffer pool allocator.");
GE_TIMESTAMP_END(BufferPoolMemoryPass, "BufferPoolMemoryPass::Run.");
// Handle parallel group .
GE_TIMESTAMP_START(ParallelGroup);
ParallelGroupPass parallel_group_pass;

File diff suppressed because it is too large Load Diff

@ -0,0 +1,136 @@
/**
* Copyright 2020 Huawei Technologies Co., Ltd
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef GE_GRAPH_PASSES_BUFFER_POOL_MEMORY_PASS_H_
#define GE_GRAPH_PASSES_BUFFER_POOL_MEMORY_PASS_H_
#include <queue>
#include "graph/graph.h"
#include "inc/graph_pass.h"
namespace ge {
class BufferPoolMemoryPass : public GraphPass {
public:
explicit BufferPoolMemoryPass() : logic_event_num_(0) {}
~BufferPoolMemoryPass() override = default;
struct BufferPool {
int64_t pool_id = 0;
int64_t pool_size = 0;
std::unordered_map<NodePtr, NodePtr> buffer_node_to_calc;
BufferPool(int64_t id, int64_t size, const std::unordered_map<NodePtr, NodePtr> &node_map)
: pool_id(id), pool_size(size), buffer_node_to_calc(node_map) {}
};
struct BufferPoolNodeItem {
NodePtr node = nullptr;
NodePtr out_calc_node = nullptr;
NodePtr pre_buffer_pool_node = nullptr;
int64_t total_size = 0;
int64_t offset_start = 0;
int64_t offset_end = 0;
bool is_last_input = true;
BufferPoolNodeItem(const NodePtr &buffer_n, const NodePtr &calc_n, const NodePtr &pre_buffer_n,
int64_t size, int64_t start, int64_t end, bool last)
: node(std::move(buffer_n)),
out_calc_node(std::move(calc_n)),
pre_buffer_pool_node(std::move(pre_buffer_n)),
total_size(size),
offset_start(start),
offset_end(end),
is_last_input(last) {}
BufferPoolNodeItem(const NodePtr &buffer_n, int64_t start, int64_t end)
: node(std::move(buffer_n)),
out_calc_node(nullptr),
pre_buffer_pool_node(nullptr),
total_size(0),
offset_start(start),
offset_end(end),
is_last_input(true) {}
};
Status Run(ComputeGraphPtr graph) override;
private:
static void ClearQueue(std::queue<std::pair<std::string, uint32_t>> &q);
static Status IsBufferPoolMemEnable(const ComputeGraphPtr &graph);
static Status CheckBufferPoolSize(int64_t total_size, int64_t pool_id, int64_t buffer_pool_size,
std::unordered_map<int64_t, int64_t> &calc_total_size);
static Status TryToFixNodeOrder(NodePtr &pre_node, NodePtr &curr_node, bool &not_change);
Status InsertMemCpyNodeAfter(ComputeGraphPtr &graph, NodePtr &node);
Status CopyOutForMultiUsedOutput(ComputeGraphPtr &graph);
Status GetBufferPoolAndPeerCalcNodes(const ComputeGraphPtr &graph);
Status SetBufferPoolSize(const std::string &batch_label, int64_t id, int64_t size);
Status AllocateAllBufferPoolSpace();
Status AllocateSpaceInBatch(const std::map<int64_t, std::vector<NodePtr>> &calc_nodes,
const std::unordered_map<int64_t, int64_t> &buffer_pool_size_map,
const std::unordered_map<NodePtr, NodePtr> &buffer_node_to_calc,
std::unordered_map<NodePtr, std::vector<BufferPoolNodeItem>> &buffer_pool_nodes_item);
Status AllocateSpaceInBufferPool(const BufferPool &buffer_pool,
const std::vector<NodePtr> &calc_nodes_in_pool,
std::unordered_map<NodePtr, std::vector<BufferPoolNodeItem>> &buffer_pool_nodes_item);
Status AllocateSpaceForBufferPoolNode(int64_t &next_start,
const BufferPool buffer_pool,
BufferPoolNodeItem &buffer_pool_node_item,
std::queue<BufferPoolNodeItem> &node_mem_range_in_pool);
NodePtr GetOffsetAndDependency(int64_t &next_start,
int64_t total_mem_size,
int64_t buffer_pool_size,
const std::unordered_map<NodePtr, NodePtr> &buffer_node_to_calc,
std::queue<BufferPoolNodeItem> &nodes_in_buffer);
Status FixTheTimingOfDependentNodes(NodePtr &dependent_calc_node, NodePtr &curr_pool_node);
uint32_t GenerateEventId(const std::string &node_name, std::queue<std::pair<std::string, uint32_t>> &event_queue);
Status SetResultOfMemoryAndEvent();
// Use map to ensure that each visit is in the order of batch label and pool id
std::map<std::string, std::map<int64_t, std::vector<NodePtr>>> calc_nodes_;
std::unordered_map<std::string, std::unordered_map<NodePtr, NodePtr>> buffer_node_to_calc_;
std::unordered_map<std::string, std::unordered_map<NodePtr, std::vector<BufferPoolNodeItem>>> peer_buffer_node_item_;
std::unordered_map<std::string, std::unordered_map<int64_t, int64_t>> buffer_pool_size_;
uint32_t logic_event_num_;
std::queue<std::pair<std::string, uint32_t>> mem_ctrl_event_;
std::queue<std::pair<std::string, uint32_t>> stream_ctrl_event_;
std::unordered_map<NodePtr, std::vector<std::string>> node_event_multiplexing_;
std::unordered_map<NodePtr, std::vector<int64_t>> buffer_node_logical_offset_;
};
} // namespace ge
#endif // GE_GRAPH_PASSES_BUFFER_POOL_MEMORY_PASS_H_

@ -43,6 +43,11 @@ rtError_t rtEventCreate(rtEvent_t *event) {
*event = new int[EVENT_LENTH];
return RT_ERROR_NONE;
}
rtError_t rtEventCreateWithFlag(rtEvent_t *event, uint32_t flag) {
return rtEventCreate(event);
}
rtError_t rtEventRecord(rtEvent_t event, rtStream_t stream) { return RT_ERROR_NONE; }
rtError_t rtEventSynchronize(rtEvent_t event) { return RT_ERROR_NONE; }

@ -276,6 +276,7 @@ set(COMMON_SRC_FILES
"${GE_CODE_DIR}/ge/graph/passes/remove_same_const_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/useless_control_out_remove_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/parallel_group_pass.cc"
"${GE_CODE_DIR}/ge/graph/passes/buffer_pool_memory_pass.cc"
"${GE_CODE_DIR}/ge/model/ge_model.cc"
"${GE_CODE_DIR}/ge/common/cust_aicpu_kernel_store.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/model_utils.cc"
@ -323,6 +324,7 @@ set(COMMON_SRC_FILES
"${GE_CODE_DIR}/ge/graph/build/memory/block_mem_assigner.cc"
"${GE_CODE_DIR}/ge/graph/build/memory/binary_block_mem_assigner.cc"
"${GE_CODE_DIR}/ge/graph/build/memory/max_block_mem_assigner.cc"
"${GE_CODE_DIR}/ge/graph/build/memory/buffer_pool_mem_assigner.cc"
"${GE_CODE_DIR}/ge/graph/manager/graph_mem_allocator.cc"
"${GE_CODE_DIR}/ge/graph/manager/graph_var_manager.cc"
"${GE_CODE_DIR}/ge/analyzer/analyzer.cc"
@ -627,6 +629,7 @@ set(SINGLE_OP_SRC_FILES
# test files
set(COMMON_TEST_FILES
"graph/passes/graph_builder_utils.cc"
"graph/utils/buffer_pool_graph_builder.cc"
"test.cc"
)
@ -703,6 +706,7 @@ set(PASS_TEST_FILES
"graph/passes/link_gen_mask_nodes_pass_unittest.cc"
"graph/passes/transpose_transdata_pass_unittest.cc"
"graph/passes/parallel_group_pass_unittest.cc"
"graph/passes/buffer_pool_memory_pass_unittest.cc"
)
set(KERNEL_TEST_FILES
@ -771,6 +775,7 @@ set(MULTI_PARTS_TEST_FILES
"graph/build/model_builder_unittest.cc"
"graph/build/mem_assigner_unittest.cc"
"graph/build/task_generator_unittest.cc"
"graph/build/buffer_pool_mem_assigner_unittest.cc"
"graph/preprocess/graph_preprocess_unittest.cc"
"graph/manager/hcom_util_unittest.cc"
"graph/manager/graph_caching_allocator_unittest.cc"

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff
Loading…
Cancel
Save