Browse Source

Merge branch 'development' of gitee.com:mindspore/graphengine into development

pull/1046/head
周莉莉 Gitee 5 years ago
parent
commit
87e4f15731
31 changed files with 503 additions and 77 deletions
  1. +1
    -0
      .gitignore
  2. +0
    -9
      ge/CMakeLists.txt
  3. +1
    -1
      ge/graph/build/run_context.cc
  4. +1
    -1
      ge/graph/load/model_manager/davinci_model.cc
  5. +53
    -2
      ge/graph/passes/multi_batch_clone_pass.cc
  6. +2
    -0
      ge/graph/passes/multi_batch_clone_pass.h
  7. +1
    -1
      ge/graph/preprocess/multi_batch_copy_graph.cc
  8. +1
    -1
      ge/graph/preprocess/multi_batch_options.cc
  9. +1
    -1
      ge/graph/preprocess/multi_batch_options.h
  10. +1
    -1
      ge/hybrid/common/tensor_value.cc
  11. +3
    -1
      ge/hybrid/executor/hybrid_model_executor.cc
  12. +43
    -2
      ge/hybrid/executor/subgraph_executor.cc
  13. +14
    -1
      ge/hybrid/executor/subgraph_executor.h
  14. +7
    -2
      ge/hybrid/model/hybrid_model.cc
  15. +7
    -1
      ge/hybrid/model/hybrid_model.h
  16. +111
    -24
      ge/hybrid/model/hybrid_model_builder.cc
  17. +2
    -0
      ge/hybrid/model/hybrid_model_builder.h
  18. +3
    -1
      ge/hybrid/node_executor/aicore/aicore_node_executor.cc
  19. +1
    -1
      ge/hybrid/node_executor/aicore/aicore_op_task.cc
  20. +3
    -0
      ge/hybrid/node_executor/aicore/aicore_op_task.h
  21. +5
    -1
      ge/hybrid/node_executor/aicore/aicore_task_builder.cc
  22. +1
    -1
      ge/hybrid/node_executor/aicore/aicore_task_builder.h
  23. +19
    -1
      ge/single_op/single_op.cc
  24. +4
    -1
      ge/single_op/single_op.h
  25. +40
    -0
      ge/single_op/single_op_model.cc
  26. +4
    -0
      ge/single_op/stream_resource.cc
  27. +1
    -0
      ge/single_op/stream_resource.h
  28. +37
    -8
      tests/depends/runtime/src/runtime_stub.cc
  29. +42
    -0
      tests/ut/ge/CMakeLists.txt
  30. +94
    -14
      tests/ut/ge/graph/load/davinci_model_unittest.cc
  31. +0
    -1
      tests/ut/ge/single_op/single_op_manager_unittest.cc

+ 1
- 0
.gitignore View File

@@ -2,6 +2,7 @@
/build
/output
/prebuilts
/cov
*.ir
*.out



+ 0
- 9
ge/CMakeLists.txt View File

@@ -639,15 +639,6 @@ set(INFER_SRC_LIST
"graph/load/model_manager/task_info/model_exit_task_info.cc"
"graph/load/model_manager/task_info/super_kernel/super_kernel_factory.cc"
"graph/load/model_manager/task_info/super_kernel/super_kernel.cc"
"single_op/task/op_task.cc"
"single_op/task/build_task_utils.cc"
"single_op/task/tbe_task_builder.cc"
"single_op/task/aicpu_task_builder.cc"
"single_op/task/aicpu_kernel_task_builder.cc"
"single_op/single_op.cc"
"single_op/single_op_model.cc"
"single_op/stream_resource.cc"
"single_op/single_op_manager.cc"
"hybrid/hybrid_davinci_model_stub.cc"
"ir_build/ge_ir_build.cc"
"ir_build/atc_ir_common.cc"


+ 1
- 1
ge/graph/build/run_context.cc View File

@@ -90,7 +90,7 @@ Status RunContextUtil::CreateRtModelResources(uint32_t stream_num, uint32_t even
// Create rt label
for (uint32_t i = 0; i < label_num; ++i) {
rtLabel_t label = nullptr;
rt_ret = rtLabelCreate(&label);
rt_ret = rtLabelCreateV2(&label, rt_model_);
if (rt_ret != RT_ERROR_NONE) {
GELOGE(RT_FAILED, "rtLabelCreate failed. rt_ret = %d, index = %u", static_cast<int>(rt_ret), i);
return RT_FAILED;


+ 1
- 1
ge/graph/load/model_manager/davinci_model.cc View File

@@ -1400,7 +1400,7 @@ Status DavinciModel::InitLabelSet(const OpDescPtr &op_desc) {
}

rtLabel_t rt_label = nullptr;
rtError_t rt_error = rtLabelCreateEx(&rt_label, stream);
rtError_t rt_error = rtLabelCreateExV2(&rt_label, rt_model_handle_, stream);
if (rt_error != RT_ERROR_NONE || rt_label == nullptr) {
GELOGE(INTERNAL_ERROR, "InitLabelSet: %s create label failed, error=0x%x.", op_desc->GetName().c_str(), rt_error);
return INTERNAL_ERROR;


+ 53
- 2
ge/graph/passes/multi_batch_clone_pass.cc View File

@@ -92,8 +92,7 @@ Status MultiBatchClonePass::Run(ComputeGraphPtr graph) {
}

// parser data dynamic info from atc parameter --input_shape
if (multibatch::ParserDataToDynmaicInfo(batch_shapes_, GetLocalOmgContext().user_input_dims,
data_to_dynamic_info_) != SUCCESS) {
if (CheckAndParseDynamicData() != SUCCESS) {
GELOGE(PARAM_INVALID, "Parse each data's own dynamic info failed");
return PARAM_INVALID;
}
@@ -177,6 +176,58 @@ Status MultiBatchClonePass::CollectIoNodes(const ComputeGraphPtr &graph) {
return SUCCESS;
}

Status MultiBatchClonePass::CheckAndParseDynamicData() {
size_t unknown_shape_count = 0;
auto data_name_and_shape = GetLocalOmgContext().user_input_dims;
std::vector<std::string> data_name_order;
for (auto &item : data_name_and_shape) {
data_name_order.push_back(item.first);
}
if (!getnext_sink_dynamic_dims_) {
for (const auto &node : all_data_nodes_) {
auto data_desc = NodeUtils::GetOutputDesc(*node, kDataOutIndex);
auto data_shape = data_desc.GetShape();
auto data_format = data_desc.GetFormat() == Format::FORMAT_NCHW ? "NCHW" :
data_desc.GetFormat() == Format::FORMAT_NHWC ? "NHWC" : "Others";
auto data_name = node->GetName();

const auto &data_shape_dims = data_shape.GetDims();
if (std::all_of(data_shape_dims.begin(), data_shape_dims.end(), [](int64_t val) { return val >= 0; })) {
continue;
}
++unknown_shape_count;
auto iter = find(data_name_order.begin(), data_name_order.end(), data_name);
if (iter == data_name_order.end()) {
if (!GetLocalOmgContext().dynamic_batch_size.empty()) {
auto ret = multibatch::CheckDynamicBatchShape(data_shape_dims, data_name);
GE_IF_BOOL_EXEC(ret == false, GELOGE(PARAM_INVALID, "Failed to check dynamic batch shape of %s.",
data_name.c_str()); return PARAM_INVALID);
} else if (!GetLocalOmgContext().dynamic_image_size.empty()) {
auto ret = multibatch::CheckDynamicImageSizeShape(data_shape_dims, data_name, data_format);
GE_IF_BOOL_EXEC(ret == false, GELOGE(PARAM_INVALID, "Failed to check dynamic image size shape of %s.",
data_name.c_str()); return PARAM_INVALID);
} else if (!GetLocalOmgContext().dynamic_dims.empty()) {
ErrorManager::GetInstance().ATCReportErrMessage("E10001", {"parameter", "reason"},
{"--input_shape", "all dynamic data must be set in --input_shape"});
GELOGE(INTERNAL_ERROR, "data: %s shape:%s must be set int --input_shape",
node->GetName().c_str(), data_shape.ToString().c_str());
return INTERNAL_ERROR;
}
data_name_and_shape.emplace_back(data_name, data_shape_dims);
}
}
}
auto ret = multibatch::ParserDataToDynamicInfo(batch_shapes_, data_name_and_shape, data_to_dynamic_info_);
GE_CHK_STATUS_RET(ret, "Failed to parse data to dynamic info.");
if (!getnext_sink_dynamic_dims_ && unknown_shape_count == 0) {
ErrorManager::GetInstance().ATCReportErrMessage("E10040");
GELOGE(PARAM_INVALID,
"Need unknow shape data when user set --dynamic_batch_size, --dynamic_image_size or --dynamic_dims");
return PARAM_INVALID;
}
return SUCCESS;
}

Status MultiBatchClonePass::InitParamsOfGetNext(const NodePtr &node) {
data_count_from_getnext_ = 0;
getnext_sink_dynamic_dims_ = false;


+ 2
- 0
ge/graph/passes/multi_batch_clone_pass.h View File

@@ -175,6 +175,8 @@ class MultiBatchClonePass : public GraphPass {
/// @return 0: SUCCESS / others: FAILED
///
Status UpdateOutputTensor(uint32_t parent_index, uint32_t unused_num);
Status CheckAndParseDynamicData();

std::string session_graph_id_;
std::vector<std::vector<int64_t>> batch_shapes_;


+ 1
- 1
ge/graph/preprocess/multi_batch_copy_graph.cc View File

@@ -738,7 +738,7 @@ Status MultiBatchGraphCopyer::CheckAndParseDynamicData(){
}
}
}
auto ret = ParserDataToDynmaicInfo(shapes_, data_name_and_shape, data_to_dynamic_info_);
auto ret = ParserDataToDynamicInfo(shapes_, data_name_and_shape, data_to_dynamic_info_);
GE_CHK_STATUS_RET(ret, "Failed to parse data to dynamic info.");
if (!getnext_sink_dynamic_dims_ && unknown_shape_count == 0) {
ErrorManager::GetInstance().ATCReportErrMessage("E10040");


+ 1
- 1
ge/graph/preprocess/multi_batch_options.cc View File

@@ -377,7 +377,7 @@ bool InitDynamicParams(vector<vector<int64_t>> &shapes) {
/// @param [out] map<string, vector<vector<int64_t>>> &data_to_dynamic_info: key:data_name. value:dynamic dims.
/// @return true: Configed for Multi batch / false: Not configed for Multi batch.
///
Status ParserDataToDynmaicInfo(const vector<vector<int64_t>> &shapes,
Status ParserDataToDynamicInfo(const vector<vector<int64_t>> &shapes,
vector<pair<string, vector<int64_t>>> &data_name_and_shape,
map<string, vector<vector<int64_t>> > &data_to_dynamic_info) {
size_t cur_data_index = 0;


+ 1
- 1
ge/graph/preprocess/multi_batch_options.h View File

@@ -74,7 +74,7 @@ Status CalcShape(const std::vector<int64_t> &batch_shape, GeShape &data_shape);
/// @param [out] map<string, vector<vector<int64_t>>> &data_to_dynamic_info: key:data_name. value:dynamic dims.
/// @return SUCCESS / PARAM_INVALID
///
Status ParserDataToDynmaicInfo(const vector<vector<int64_t>> &shapes,
Status ParserDataToDynamicInfo(const vector<vector<int64_t>> &shapes,
vector<pair<string, vector<int64_t>>> &data_name_and_shape,
map<string, vector<vector<int64_t>>> &data_to_dynamic_info);



+ 1
- 1
ge/hybrid/common/tensor_value.cc View File

@@ -71,7 +71,7 @@ TensorValue::TensorValue(void *buffer, size_t size) : ref_buffer_(buffer), ref_s
TensorValue::~TensorValue() { Destroy(); }

void TensorValue::Destroy() {
if (buffer_ != nullptr || ref_buffer_ != nullptr) {
if (buffer_ != nullptr) {
GELOGD("Unref tensor: %s", DebugString().c_str());
buffer_.reset();
}


+ 3
- 1
ge/hybrid/executor/hybrid_model_executor.cc View File

@@ -71,12 +71,14 @@ Status HybridModelExecutor::ExecuteGraphInternal(SubgraphExecutor &executor,
GE_CHK_STATUS_RET_NOLOG(ResetExecutionContext(context_));
RECORD_MODEL_EXECUTION_EVENT(&context_, "[InitContext] End");

HYBRID_CHK_STATUS_RET(executor.ExecuteAsync(args.inputs, args.input_desc), "Failed to execute partitioned call.");
HYBRID_CHK_STATUS_RET(executor.ExecuteAsync(args.inputs, args.input_desc, args.outputs),
"Failed to execute partitioned call.");
RECORD_MODEL_EXECUTION_EVENT(&context_, "[ExecuteAsync] End");

HYBRID_CHK_STATUS_RET(executor.Synchronize(), "Failed to sync root graph.");
RECORD_MODEL_EXECUTION_EVENT(&context_, "[Synchronize] End");

args.outputs.clear();
HYBRID_CHK_STATUS_RET(executor.GetOutputs(args.outputs, args.output_desc), "Failed to get outputs");
RECORD_MODEL_EXECUTION_EVENT(&context_, "[GetOutput] End");
return SUCCESS;


+ 43
- 2
ge/hybrid/executor/subgraph_executor.cc View File

@@ -131,10 +131,14 @@ Status SubgraphExecutor::InitInputsForKnownShape(const std::vector<TensorValue>
}

Status SubgraphExecutor::ExecuteAsync(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc) {
const std::vector<ConstGeTensorDescPtr> &input_desc,
const std::vector<TensorValue> &outputs) {
GELOGD("[%s] is dynamic = %s", graph_item_->GetName().c_str(), graph_item_->IsDynamic() ? "true" : "false");
GE_CHK_STATUS_RET(Init(inputs, input_desc), "[%s] Failed to init executor.", graph_item_->GetName().c_str());

if (!outputs.empty()) {
GE_CHK_STATUS_RET(EnableOutputZeroCopy(outputs),
"Failed to enable output zero copy by user provided outputs.");
}
if (!graph_item_->IsDynamic()) {
return ExecuteAsyncForKnownShape(inputs);
}
@@ -144,6 +148,11 @@ Status SubgraphExecutor::ExecuteAsync(const std::vector<TensorValue> &inputs,
return SUCCESS;
}

Status SubgraphExecutor::ExecuteAsync(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc) {
return ExecuteAsync(inputs, input_desc, {});
}

Status SubgraphExecutor::ExecuteAsyncForKnownShape(const std::vector<TensorValue> &inputs) {
GELOGD("[%s] subgraph is not dynamic.", graph_item_->GetName().c_str());
if (graph_item_->GetAllNodes().size() != 1) {
@@ -440,5 +449,37 @@ Status SubgraphExecutor::SetOutputsToParentNode(TaskContext &task_context) {

return SUCCESS;
}

Status SubgraphExecutor::EnableOutputZeroCopy(const vector<TensorValue> &outputs) {
GELOGD("To enable zero copy, output number = %zu", outputs.size());
const auto &output_edges = graph_item_->GetOutputEdges();
// Op -> MetOutput, set the output tensor of Op that output to the NetOutput node
if (outputs.size() != output_edges.size()) {
GELOGE(PARAM_INVALID, "Output number mismatches, expect = %zu, but given = %zu",
output_edges.size(),
outputs.size());
return PARAM_INVALID;
}

for (size_t i = 0; i < outputs.size(); ++i) {
auto &output_tensor = outputs[i];
auto &output_node = output_edges[i].first;
int output_idx = output_edges[i].second;
GELOGD("[%s] Set output tensor[%zu] to [%s]'s output[%d], tensor = %s",
graph_item_->GetName().c_str(),
i,
output_node->NodeName().c_str(),
output_idx,
output_tensor.DebugString().c_str());

GE_CHK_STATUS_RET(subgraph_context_->SetOutput(*output_node, output_idx, output_tensor),
"[%s] Failed to set input tensor[%zu]",
graph_item_->GetName().c_str(),
i);
}

GELOGD("Done enabling zero copy for outputs successfully.");
return SUCCESS;
}
} // namespace hybrid
} // namespace ge

+ 14
- 1
ge/hybrid/executor/subgraph_executor.h View File

@@ -43,7 +43,19 @@ class SubgraphExecutor {
* @param input_desc input tensor descriptions
* @return SUCCESS on success, error code otherwise
*/
Status ExecuteAsync(const std::vector<TensorValue> &inputs, const std::vector<ConstGeTensorDescPtr> &input_desc);
Status ExecuteAsync(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc);

/**
* Execute subgraph async, output tensor address(not data) and output tensor descriptions are
* valid after this method returned
* @param inputs input tensors
* @param input_desc input tensor descriptions
* @return SUCCESS on success, error code otherwise
*/
Status ExecuteAsync(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc,
const std::vector<TensorValue> &outputs);

/**
* Execute subgraph async, output tensor address(not data) and output tensor descriptions are
@@ -76,6 +88,7 @@ class SubgraphExecutor {

private:
Status PrepareForExecution(GraphExecutionContext *ctx, NodeState &node_state);
Status EnableOutputZeroCopy(const std::vector<TensorValue> &outputs);
static Status InferShape(ShapeInferenceEngine *shape_inference_engine, NodeState &node_state);
Status Init(const std::vector<TensorValue> &inputs,
const std::vector<ConstGeTensorDescPtr> &input_desc);


+ 7
- 2
ge/hybrid/model/hybrid_model.cc View File

@@ -40,9 +40,14 @@ HybridModel::~HybridModel() {
GELOGD("[%s] HybridModel destroyed.", model_name_.c_str());
}

Status HybridModel::Init() {
Status HybridModel::Init(bool is_single_op) {
GELOGD("Start to init hybrid model.");
GE_CHK_STATUS_RET(HybridModelBuilder(*this).Build(), "Failed to build hybrid model.");
is_single_op_ = is_single_op;
if (is_single_op) {
GE_CHK_STATUS_RET(HybridModelBuilder(*this).BuildForSingleOp(), "Failed to build hybrid model.");
} else {
GE_CHK_STATUS_RET(HybridModelBuilder(*this).Build(), "Failed to build hybrid model.");
}
GELOGD("HybridModel initialized successfully.");
return SUCCESS;
}


+ 7
- 1
ge/hybrid/model/hybrid_model.h View File

@@ -37,7 +37,7 @@ class HybridModel {

~HybridModel();

Status Init();
Status Init(bool is_single_op = false);

const NodeItem *GetNodeItem(const NodePtr &node) const;

@@ -69,6 +69,10 @@ class HybridModel {
return model_id_;
}

bool IsSingleOp() const {
return is_single_op_;
}

TensorValue* GetVariable(const string &name) const;

NodePtr GetVariableNode(const string &name) const;
@@ -131,11 +135,13 @@ class HybridModel {
std::map<NodePtr, std::unique_ptr<NodeItem>> node_items_;

bool is_new_model_desc_ = false; // support aipp
bool is_single_op_ = false;

// runtime fields
uint32_t device_id_ = 0;
uint32_t model_id_ = 0;
uint8_t *var_mem_base_ = nullptr;
std::unique_ptr<TensorBuffer> weight_buffer_;
RuntimeParam root_runtime_param_;
};
} // namespace hybrid


+ 111
- 24
ge/hybrid/model/hybrid_model_builder.cc View File

@@ -147,6 +147,21 @@ Status HybridModelBuilder::Build() {
return SUCCESS;
}

Status HybridModelBuilder::BuildForSingleOp() {
GE_CHK_STATUS_RET(ValidateParams(), "Failed to validate GeRootModel");
hybrid_model_.model_name_ = ge_root_model_->GetRootGraph()->GetName();
GELOGI("[%s] Start to build hybrid model.", GetGraphName());
auto ret = ge_root_model_->GetSubgraphInstanceNameToModel();
const GeModelPtr ge_model = ret[ge_root_model_->GetRootGraph()->GetName()];
GE_CHK_STATUS_RET(IndexTaskDefs(ge_root_model_->GetRootGraph(), ge_model),
"[%s] Failed to index task defs", GetGraphName());
GE_CHK_STATUS_RET(LoadGraph(), "[%s] Failed to load graph", GetGraphName());
GE_CHK_STATUS_RET(InitWeights(), "[%s] Failed to init weights", GetGraphName());
GE_CHK_STATUS_RET(LoadTasks(), "[%s] Failed to load tasks", GetGraphName());
GELOGI("[%s] Done building hybrid model for single op successfully.", GetGraphName());
return SUCCESS;
}

Status HybridModelBuilder::ValidateParams() {
GE_CHECK_NOTNULL(ge_root_model_);
GE_CHECK_NOTNULL(ge_root_model_->GetRootGraph());
@@ -951,46 +966,71 @@ Status HybridModelBuilder::InitVariableTensors() {
}

Status HybridModelBuilder::InitWeights() {
// For constant in root graph
const auto &root_graph = ge_root_model_->GetRootGraph();
const auto &subgraph_models = ge_root_model_->GetSubgraphInstanceNameToModel();
auto iter = subgraph_models.find(root_graph->GetName());
if (iter == subgraph_models.end()) {
GELOGD("Root graph model not found");
return SUCCESS;
}

auto &root_model = iter->second;
const auto &weight_buffer = root_model->GetWeight();
if (weight_buffer.GetSize() == 0) {
GELOGD("weight is empty");
return SUCCESS;
}

auto allocator = NpuMemoryAllocator::GetAllocator();
GE_CHECK_NOTNULL(allocator);

for (auto &it : hybrid_model_.node_items_) {
auto &node_item = it.second;
if (node_item->node_type != CONSTANT) {
hybrid_model_.weight_buffer_ = TensorBuffer::Create(allocator, weight_buffer.size());
GE_CHECK_NOTNULL(hybrid_model_.weight_buffer_);
auto weight_base = reinterpret_cast<uint8_t *>(hybrid_model_.weight_buffer_->GetData());
GE_CHK_RT_RET(rtMemcpy(weight_base,
hybrid_model_.weight_buffer_->GetSize(),
weight_buffer.GetData(),
weight_buffer.GetSize(),
RT_MEMCPY_HOST_TO_DEVICE));

GELOGI("Init weight mem successfully, weight base %p, weight size = %zu",
weight_base,
hybrid_model_.weight_buffer_->GetSize());
for (auto &node : root_graph->GetDirectNode()) {
if (node->GetType() != CONSTANT) {
continue;
}

const auto &constant_node = node_item->node;
auto op_desc = constant_node->GetOpDesc();
auto op_desc = node->GetOpDesc();
auto v_weights = ModelUtils::GetWeights(op_desc);
if (v_weights.empty()) {
GELOGE(INTERNAL_ERROR, "[%s] Constant has no value", constant_node->GetName().c_str());
GELOGE(INTERNAL_ERROR, "[%s] Constant has no value", node->GetName().c_str());
return INTERNAL_ERROR;
}
auto *ge_tensor = const_cast<GeTensor *>(v_weights[0].get());
auto output_desc = op_desc->MutableOutputDesc(0);
GE_CHECK_NOTNULL(output_desc);
auto tensor_size = ge_tensor->GetData().GetSize();
GELOGD("[%s] Start to init Constant node [%s], size = %ld",
GE_CHECK_NOTNULL(ge_tensor);
const GeTensorDesc &tensor_desc = ge_tensor->GetTensorDesc();
int64_t tensor_size = 0;
GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetSize(*op_desc->MutableOutputDesc(0), tensor_size),
"[%s] Failed to get tensor size",
node->GetName().c_str());
int64_t data_offset = 0;
GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetDataOffset(tensor_desc, data_offset),
"[%s] Failed to get data offset",
node->GetName().c_str());
GELOGD("[%s] Start to init Constant node [%s], size = %ld, offset = %ld",
GetGraphName(),
constant_node->GetName().c_str(),
tensor_size);
node->GetName().c_str(),
tensor_size,
data_offset);

auto tensor_buffer = TensorBuffer::Create(allocator, tensor_size);
auto tensor_buffer = TensorBuffer::Create(weight_base + data_offset, tensor_size);
GE_CHECK_NOTNULL(tensor_buffer);
std::unique_ptr<TensorValue> constant_tensor(new (std::nothrow)TensorValue(std::move(tensor_buffer)));
GE_CHECK_NOTNULL(constant_tensor);
constant_tensor->SetName("Constant_" + op_desc->GetName());
if (tensor_size > 0) {
GE_CHK_RT_RET(rtMemcpy(constant_tensor->MutableData(),
constant_tensor->GetSize(),
ge_tensor->GetData().data(),
ge_tensor->GetData().size(),
RT_MEMCPY_HOST_TO_DEVICE));
}

hybrid_model_.constant_tensors_.emplace(constant_node, std::move(constant_tensor));
GELOGD("[%s] Constant node [%s] added, size = %ld", GetGraphName(), constant_node->GetName().c_str(), tensor_size);
hybrid_model_.constant_tensors_.emplace(node, std::move(constant_tensor));
GELOGD("[%s] Constant node [%s] added, size = %ld", GetGraphName(), node->GetName().c_str(), tensor_size);
}
return SUCCESS;
}
@@ -1038,6 +1078,53 @@ Status HybridModelBuilder::LoadGeModel(ComputeGraph &sub_graph, const GeModelPtr
return SUCCESS;
}

Status HybridModelBuilder::IndexTaskDefs(const ComputeGraphPtr &sub_graph, const GeModelPtr &ge_model) {
// index task defs
GELOGD("To index tasks for subgraph: %s", sub_graph->GetName().c_str());
std::unordered_map<int64_t, NodePtr> node_map;
for (const auto &node : sub_graph->GetDirectNode()) {
GE_CHECK_NOTNULL(node);
GE_CHECK_NOTNULL(node->GetOpDesc());
auto node_id = node->GetOpDesc()->GetId();
GELOGD("op_index = %ld, node_name = %s", node_id, node->GetName().c_str());
node_map.emplace(node_id, node);
}

auto tasks = ge_model->GetModelTaskDefPtr()->task();
for (int i = 0; i < tasks.size(); ++i) {
const domi::TaskDef &task_def = tasks[i];
GELOGI("Task id = %d, task type = %d", i, task_def.type());
auto task_type = static_cast<rtModelTaskType_t>(task_def.type());
uint32_t op_index = -1;
if (task_type == RT_MODEL_TASK_KERNEL) {
op_index = task_def.kernel().context().op_index();
} else if (task_type == RT_MODEL_TASK_KERNEL_EX) {
op_index = task_def.kernel_ex().op_index();
} else if (task_type == RT_MODEL_TASK_HCCL) {
op_index = task_def.kernel_hccl().op_index();
} else {
GELOGD("Skip task type: %d", static_cast<int>(task_type));
continue;
}

auto iter = node_map.find(op_index);
if (iter == node_map.end()) {
GELOGE(INTERNAL_ERROR, "Failed to get node by index = %u", op_index);
return INTERNAL_ERROR;
}

auto &node = iter->second;
if (task_type == RT_MODEL_TASK_KERNEL) {
ge_model->GetTBEKernelStore().LoadTBEKernelBinToOpDesc(node->GetOpDesc());
}

GELOGD("Task loaded for node: %s, task type = %d, op_index = %u", node->GetName().c_str(), task_type, op_index);
hybrid_model_.task_defs_[node].emplace_back(task_def);
}

return SUCCESS;
}

Status HybridModelBuilder::IndexTaskDefs() {
const auto &root_graph = ge_root_model_->GetRootGraph();
if (SetOutputNameAttr(*root_graph) != SUCCESS) {


+ 2
- 0
ge/hybrid/model/hybrid_model_builder.h View File

@@ -35,6 +35,7 @@ class HybridModelBuilder {
explicit HybridModelBuilder(HybridModel &hybrid_model);
~HybridModelBuilder() = default;
Status Build();
Status BuildForSingleOp();

private:
static Status UpdateAnchorStatus(const NodePtr &node);
@@ -64,6 +65,7 @@ class HybridModelBuilder {
Status ParseDependentInputNodes(NodeItem &node_item, const std::vector<string> &dependencies);
Status ParseDependentForFusedSubgraph(NodeItem &node_item);
Status IndexTaskDefs();
Status IndexTaskDefs(const ComputeGraphPtr &sub_graph, const GeModelPtr &ge_model);
Status IndexSpecialNodes();
Status InitRuntimeParams();
Status InitModelMem();


+ 3
- 1
ge/hybrid/node_executor/aicore/aicore_node_executor.cc View File

@@ -49,6 +49,7 @@ Status AiCoreNodeExecutor::Initialize() {
Status AiCoreNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr<NodeTask> &task) const {
GE_CHECK_NOTNULL(node);
GELOGI("AiCoreNodeExecutor(%s) LoadTask Start.", node->GetName().c_str());
bool is_single_op = model.IsSingleOp();

auto *task_defs = model.GetTaskDefs(node);
if (task_defs == nullptr || task_defs->empty()) {
@@ -66,7 +67,8 @@ Status AiCoreNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &nod

AiCoreTaskBuilder builder(node->GetOpDesc(), *task_defs);
std::unique_ptr<NodeTask> node_task;
GE_CHK_STATUS_RET(builder.BuildTask(node_task, true), "[%s] Failed to build op tasks.", node->GetName().c_str());
GE_CHK_STATUS_RET(builder.BuildTask(node_task, true, is_single_op),
"[%s] Failed to build op tasks.", node->GetName().c_str());
task = std::move(node_task);
GELOGI("AiCoreNodeExecutor(%s) LoadTask End.", node->GetName().c_str());
return SUCCESS;


+ 1
- 1
ge/hybrid/node_executor/aicore/aicore_op_task.cc View File

@@ -65,7 +65,7 @@ Status AiCoreOpTask::RegisterTbeHandle(const OpDesc &op_desc) {
}
TBEHandleStore &kernel_store = TBEHandleStore::GetInstance();
rtError_t rt_ret = rtQueryFunctionRegistered(stub_name_.c_str());
if (rt_ret != RT_ERROR_NONE) {
if (rt_ret != RT_ERROR_NONE || is_single_op_) {
void *bin_handle = nullptr;
if (!kernel_store.FindTBEHandle(stub_name_.c_str(), bin_handle)) {
GELOGI("TBE: can't find the kernel_name[%s] in HandleMap", stub_name_.c_str());


+ 3
- 0
ge/hybrid/node_executor/aicore/aicore_op_task.h View File

@@ -50,6 +50,8 @@ class AiCoreOpTask {

uint32_t GetBlockDim() const {return block_dim_;}

void SetSingleOp(bool is_single_op) {is_single_op_ = is_single_op;};

protected:
Status UpdateTilingInfo(TaskContext &context);
virtual std::string GetKeyForOpParamSize() const;
@@ -72,6 +74,7 @@ class AiCoreOpTask {
uint32_t args_size_ = 0;
uint32_t block_dim_ = 1;
bool clear_atomic_ = true;
bool is_single_op_ = false;
std::vector<int> output_indices_to_skip_;
};



+ 5
- 1
ge/hybrid/node_executor/aicore/aicore_task_builder.cc View File

@@ -37,7 +37,9 @@ AiCoreTaskBuilder::AiCoreTaskBuilder(const OpDescPtr &op_desc, const std::vector
: op_desc_(op_desc), task_defs_(task_defs) {
}

Status AiCoreTaskBuilder::BuildTask(std::unique_ptr<NodeTask> &node_task, bool ignore_failure_on_atomic) {
Status AiCoreTaskBuilder::BuildTask(std::unique_ptr<NodeTask> &node_task,
bool ignore_failure_on_atomic,
bool is_single_op) {
GE_CHECK_NOTNULL(op_desc_);
if (task_defs_.size() > kNumTaskWithAtomicAddrCleanTask) {
GELOGE(INTERNAL_ERROR,
@@ -68,6 +70,7 @@ Status AiCoreTaskBuilder::BuildTask(std::unique_ptr<NodeTask> &node_task, bool i
auto atomic_task =
std::unique_ptr<AtomicAddrCleanOpTask>(new(std::nothrow)AtomicAddrCleanOpTask());
GE_CHECK_NOTNULL(atomic_task);
atomic_task->SetSingleOp(is_single_op);
GE_CHK_STATUS_RET(atomic_task->Init(*op_desc_, task_defs_.front()),
"[%s] Failed to init task for AtomicAddrClean",
op_desc_->GetName().c_str());
@@ -77,6 +80,7 @@ Status AiCoreTaskBuilder::BuildTask(std::unique_ptr<NodeTask> &node_task, bool i
// build aicore task
auto aicore_task = std::unique_ptr<AiCoreOpTask>(new(std::nothrow)AiCoreOpTask());
GE_CHECK_NOTNULL(aicore_task);
aicore_task->SetSingleOp(is_single_op);
GE_CHK_STATUS_RET(aicore_task->Init(*op_desc_, task_defs_.back()),
"[%s] Failed to init task for AtomicAddrClean",
op_desc_->GetName().c_str());


+ 1
- 1
ge/hybrid/node_executor/aicore/aicore_task_builder.h View File

@@ -47,7 +47,7 @@ class AiCoreTaskBuilder {
AiCoreTaskBuilder(const OpDescPtr &op_desc, const std::vector<domi::TaskDef> &task_defs);
~AiCoreTaskBuilder() = default;

Status BuildTask(std::unique_ptr<NodeTask> &node_task, bool ignore_failure_on_atomic);
Status BuildTask(std::unique_ptr<NodeTask> &node_task, bool ignore_failure_on_atomic, bool is_single_op = false);

private:
bool ExpectAtomicAddrCleanTask();


+ 19
- 1
ge/single_op/single_op.cc View File

@@ -256,9 +256,27 @@ Status DynamicSingleOp::ExecuteAsync(const vector<GeTensorDesc> &input_desc,
const vector<DataBuffer> &input_buffers,
vector<GeTensorDesc> &output_desc,
vector<DataBuffer> &output_buffers) {
GE_CHECK_NOTNULL(op_task_);
GE_CHK_STATUS_RET_NOLOG(ValidateParams(input_desc, input_buffers, output_desc, output_buffers));
if (hybrid_model_executor_ != nullptr) {
GELOGD("Execute multi-task dynamic single op by hybrid model executor");
hybrid::HybridModelExecutor::ExecuteArgs args;
for (auto &input : input_buffers) {
args.inputs.emplace_back(hybrid::TensorValue(input.data, input.length));
}
for (auto &output : output_buffers) {
args.outputs.emplace_back(hybrid::TensorValue(output.data, output.length));
}
for (auto &tensor_desc : input_desc) {
auto desc = MakeShared<GeTensorDesc>(tensor_desc);
GE_CHECK_NOTNULL(desc);
args.input_desc.emplace_back(desc);
}

return hybrid_model_executor_->Execute(args);
}

std::lock_guard<std::mutex> lk(*stream_mutex_);
GE_CHECK_NOTNULL(op_task_);

GE_CHK_STATUS_RET_NOLOG(op_task_->LaunchKernel(input_desc, input_buffers, output_desc, output_buffers, stream_));
GE_CHK_STATUS_RET_NOLOG(ProfilingTaskInfo(op_task_.get(), kShapeTypeDynamic));


+ 4
- 1
ge/single_op/single_op.h View File

@@ -28,6 +28,7 @@
#include "runtime/stream.h"
#include "task/op_task.h"
#include "cce/aicpu_engine_struct.h"
#include "hybrid/executor/hybrid_model_executor.h"

namespace ge {
class StreamResource;
@@ -46,7 +47,7 @@ class SingleOp {
Status GetArgs(const std::vector<DataBuffer> &inputs, const std::vector<DataBuffer> &outputs);

friend class SingleOpModel;
StreamResource *stream_resource_;
StreamResource *stream_resource_ = nullptr;
std::mutex *stream_mutex_;
rtStream_t stream_ = nullptr;
std::vector<void *> input_addr_list_;
@@ -77,6 +78,8 @@ class DynamicSingleOp {
std::vector<DataBuffer> &outputs) const;

std::unique_ptr<OpTask> op_task_;
std::unique_ptr<hybrid::HybridModel> hybrid_model_;
std::unique_ptr<hybrid::HybridModelExecutor> hybrid_model_executor_;
uintptr_t resource_id_ = 0;
std::mutex *stream_mutex_;
rtStream_t stream_ = nullptr;


+ 40
- 0
ge/single_op/single_op_model.cc View File

@@ -31,6 +31,8 @@
#include "task/aicpu_task_builder.h"
#include "task/aicpu_kernel_task_builder.h"
#include "task/tbe_task_builder.h"
#include "hybrid/executor/hybrid_model_executor.h"
#include "hybrid/node_executor/node_executor.h"

static std::atomic<std::uint64_t> aicpu_kernel_id(0);

@@ -42,6 +44,20 @@ namespace ge {
namespace {
const size_t kDataOutputNum = 1;
} // namespace
static Status IfInferDepend(GeModelPtr &ge_model, bool &flag) {
auto comp_graph = GraphUtils::GetComputeGraph(ge_model->GetGraph());
for (const auto &node : comp_graph->GetAllNodes()) {
auto op_desc = node->GetOpDesc();
GE_CHECK_NOTNULL(op_desc);
const auto &depends = op_desc->GetOpInferDepends();
if (!depends.empty()) {
flag = true;
return SUCCESS;
}
}
return SUCCESS;
}

SingleOpModel::SingleOpModel(const std::string &model_name, const void *model_data, uint32_t model_size)
: model_name_(model_name), ori_model_data_(model_data), ori_model_size_(model_size) {}

@@ -478,6 +494,30 @@ Status SingleOpModel::BuildDynamicOp(StreamResource &resource, DynamicSingleOp &
single_op.num_outputs_ = netoutput_op_->GetAllInputsSize();
GE_CHK_STATUS_RET_NOLOG(InitModelMem(resource));
model_params_.memory_size = UINT_MAX;

auto ge_model = model_helper_.GetGeModel();
GE_CHECK_NOTNULL(ge_model);
bool infer_depend_flag = false;
GE_CHK_STATUS_RET_NOLOG(IfInferDepend(ge_model, infer_depend_flag));
if (ge_model->GetModelTaskDefPtr()->task_size() > 1 || infer_depend_flag) {
GELOGD("Build single op HybridModel.");
GE_CHK_STATUS_RET_NOLOG(hybrid::NodeExecutorManager::GetInstance().EnsureInitialized());
auto root_model = model_helper_.GetGeRootModel();
GE_CHECK_NOTNULL(root_model);
root_model->SetRootGraph(GraphUtils::GetComputeGraph(ge_model->GetGraph()));
root_model->SetSubgraphInstanceNameToModel(root_model->GetRootGraph()->GetName(), ge_model);
single_op.hybrid_model_.reset(new (std::nothrow)hybrid::HybridModel(root_model));
GE_CHECK_NOTNULL(single_op.hybrid_model_);
GE_CHK_STATUS_RET(single_op.hybrid_model_->Init(true), "Failed to init hybrid model");
int32_t device_id = 0;
GE_CHK_RT_RET(rtGetDevice(&device_id));
single_op.hybrid_model_executor_.reset(new (std::nothrow)hybrid::HybridModelExecutor(single_op.hybrid_model_.get(),
device_id,
resource.GetStream()));
GE_CHECK_NOTNULL(single_op.hybrid_model_executor_);
GE_CHK_STATUS_RET(single_op.hybrid_model_executor_->Init(), "Failed to init hybrid model");
return SUCCESS;
}
return BuildTaskListForDynamicOp(single_op);
}
} // namespace ge

+ 4
- 0
ge/single_op/stream_resource.cc View File

@@ -61,6 +61,10 @@ DynamicSingleOp *StreamResource::GetDynamicOperator(const void *key) {
return it->second.get();
}

rtStream_t StreamResource::GetStream() const {
return stream_;
}

void StreamResource::SetStream(rtStream_t stream) {
stream_ = stream;
}


+ 1
- 0
ge/single_op/stream_resource.h View File

@@ -37,6 +37,7 @@ class StreamResource {
StreamResource(StreamResource &&) = delete;
StreamResource &operator=(const StreamResource &) = delete;
StreamResource &operator=(StreamResource &&) = delete;
rtStream_t GetStream() const;
void SetStream(rtStream_t stream);

SingleOp *GetOperator(const void *key);


+ 37
- 8
tests/depends/runtime/src/runtime_stub.cc View File

@@ -245,9 +245,35 @@ rtError_t rtProfilerInit(const char *prof_dir, const char *address, const char *

rtError_t rtProfilerStart(void) { return RT_ERROR_NONE; }

rtError_t rtLabelCreate(rtLabel_t *label) { return RT_ERROR_NONE; }
rtError_t rtLabelCreate(rtLabel_t *label) {
*label = new uint64_t;
return RT_ERROR_NONE;
}

rtError_t rtLabelCreateEx(rtLabel_t *label, rtStream_t stream) {
*label = new uint64_t;
return RT_ERROR_NONE;
}

rtError_t rtLabelCreateV2(rtLabel_t *label, rtModel_t model) {
*label = new uint64_t;
return RT_ERROR_NONE;
}

rtError_t rtLabelDestroy(rtLabel_t label) { return RT_ERROR_NONE; }
rtError_t rtLabelCreateExV2(rtLabel_t *label, rtModel_t model, rtStream_t stream) {
*label = new uint64_t;
return RT_ERROR_NONE;
}

rtError_t rtLabelListCpy(rtLabel_t *label, uint32_t labelNumber, void *dst, uint32_t dstMax) {
return RT_ERROR_NONE;
}

rtError_t rtLabelDestroy(rtLabel_t label) {
uint64_t *stub = static_cast<uint64_t *>(label);
delete stub;
return RT_ERROR_NONE;
}

rtError_t rtLabelSet(rtLabel_t label, rtStream_t stream) { return RT_ERROR_NONE; }

@@ -255,8 +281,17 @@ rtError_t rtLabelSwitch(void *ptr, rtCondition_t condition, uint32_t value, rtLa
return RT_ERROR_NONE;
}

rtError_t rtLabelSwitchByIndex(void *ptr, uint32_t max, void *labelInfoPtr, rtStream_t stream) {
return RT_ERROR_NONE;
}

rtError_t rtLabelGoto(rtLabel_t label, rtStream_t stream) { return RT_ERROR_NONE; }

rtError_t rtLabelGotoEx(rtLabel_t label, rtStream_t stream) {
return RT_ERROR_NONE;
}


rtError_t rtInvalidCache(uint64_t base, uint32_t len) { return RT_ERROR_NONE; }

rtError_t rtModelLoadComplete(rtModel_t model) { return RT_ERROR_NONE; }
@@ -364,12 +399,6 @@ rtError_t rtSetCtxINFMode(bool mode)
return RT_ERROR_NONE;
}

rtError_t rtLabelCreateEx(rtLabel_t *label, rtStream_t stream)
{
*label = new uint32_t;
return RT_ERROR_NONE;
}

rtError_t rtGetRtCapability(rtFeatureType_t featureType, int32_t featureInfo, int64_t *value)
{
return RT_ERROR_NONE;


+ 42
- 0
tests/ut/ge/CMakeLists.txt View File

@@ -404,6 +404,8 @@ set(DISTINCT_GRAPH_LOAD_SRC_FILES
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/kernel_ex_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/kernel_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/label_set_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/label_goto_ex_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/label_switch_by_index_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/memcpy_addr_async_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/memcpy_async_task_info.cc"
"${GE_CODE_DIR}/ge/graph/load/model_manager/task_info/profiler_trace_task_info.cc"
@@ -562,6 +564,46 @@ set(SINGLE_OP_SRC_FILES
"${GE_CODE_DIR}/ge/single_op/single_op_manager.cc"
"${GE_CODE_DIR}/ge/single_op/task/aicpu_task_builder.cc"
"${GE_CODE_DIR}/ge/single_op/task/aicpu_kernel_task_builder.cc"
"${GE_CODE_DIR}/ge/hybrid/common/tensor_value.cc"
"${GE_CODE_DIR}/ge/hybrid/common/npu_memory_allocator.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/rt_callback_manager.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/node_state.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/node_done_manager.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/hybrid_profiler.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/hybrid_model_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/hybrid_model_async_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/hybrid_execution_context.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/subgraph_context.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/subgraph_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/worker/task_compile_engine.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/worker/shape_inference_engine.cc"
"${GE_CODE_DIR}/ge/hybrid/executor/worker/execution_engine.cc"
"${GE_CODE_DIR}/ge/hybrid/model/hybrid_model.cc"
"${GE_CODE_DIR}/ge/hybrid/model/hybrid_model_builder.cc"
"${GE_CODE_DIR}/ge/hybrid/model/node_item.cc"
"${GE_CODE_DIR}/ge/hybrid/model/graph_item.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/aicore/aicore_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/aicore/aicore_op_task.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/aicore/aicore_task_builder.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/aicore/aicore_task_compiler.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/aicpu/aicpu_ext_info.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/aicpu/aicpu_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/compiledsubgraph/known_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/ge_local/ge_local_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/host_cpu_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/kernel_factory.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/kernel/no_op_kernel.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/kernel/variable_kernel.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/kernel/assign_kernel.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/kernel/random_uniform_kernel.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/host_cpu/kernel/data_kernel.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/controlop/control_op_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/partitioned_call/partitioned_call_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/hccl/hccl_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/rts/rts_node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/node_executor.cc"
"${GE_CODE_DIR}/ge/hybrid/node_executor/task_context.cc"
"${GE_CODE_DIR}/ge/hybrid/hybrid_davinci_model.cc"
)

# test files


+ 94
- 14
tests/ut/ge/graph/load/davinci_model_unittest.cc View File

@@ -32,21 +32,9 @@ class UtestDavinciModel : public testing::Test {
void SetUp() {}

void TearDown() {}
public:
NodePtr MakeNode(const ComputeGraphPtr &graph, uint32_t in_num, uint32_t out_num, string name, string type) {
GeTensorDesc test_desc(GeShape(), FORMAT_NCHW, DT_FLOAT);
auto op_desc = std::make_shared<OpDesc>(name, type);
for (auto i = 0; i < in_num; ++i) {
op_desc->AddInputDesc(test_desc);
}
for (auto i = 0; i < out_num; ++i) {
op_desc->AddOutputDesc(test_desc);
}
return graph->AddNode(op_desc);
}
};

/*TEST_F(UtestDavinciModel, init_success) {
TEST_F(UtestDavinciModel, init_success) {
DavinciModel model(0, nullptr);
ComputeGraphPtr graph = make_shared<ComputeGraph>("default");
ProfilingManager::Instance().is_load_profiling_ = true;
@@ -130,7 +118,7 @@ class UtestDavinciModel : public testing::Test {
EXPECT_EQ(outputs.size(), 1);

ProfilingManager::Instance().is_load_profiling_ = false;
}*/
}

TEST_F(UtestDavinciModel, init_data_op) {
DavinciModel model(0, nullptr);
@@ -770,4 +758,96 @@ TEST_F(UtestDavinciModel, LoadWithQueue_fail_with_diff_args) {
model.output_data_info_[0] = zero_copy_offset;
EXPECT_EQ(model.LoadWithQueue(), INTERNAL_ERROR);
}

// test label_set_task Init
TEST_F(UtestDavinciModel, label_task_success) {
DavinciModel model(0, nullptr);
ComputeGraphPtr graph = make_shared<ComputeGraph>("default");

GeModelPtr ge_model = make_shared<GeModel>();
ge_model->SetGraph(GraphUtils::CreateGraphFromComputeGraph(graph));
AttrUtils::SetInt(ge_model, ATTR_MODEL_MEMORY_SIZE, 5120000);
AttrUtils::SetInt(ge_model, ATTR_MODEL_STREAM_NUM, 1);

shared_ptr<domi::ModelTaskDef> model_task_def = make_shared<domi::ModelTaskDef>();
ge_model->SetModelTaskDef(model_task_def);

GeTensorDesc tensor(GeShape(), FORMAT_ND, DT_INT32);
TensorUtils::SetSize(tensor, 64);

uint32_t op_index = 0;
{
OpDescPtr op_desc = CreateOpDesc("label_switch", LABELSWITCHBYINDEX);
op_desc->AddInputDesc(tensor);
op_desc->SetInputOffset({1024});
NodePtr node = graph->AddNode(op_desc); // op_index = 0
EXPECT_TRUE(AttrUtils::SetListInt(op_desc, ATTR_NAME_LABEL_SWITCH_LIST, {0, 1}));

domi::TaskDef *task_def1 = model_task_def->add_task();
task_def1->set_stream_id(0);
task_def1->set_type(RT_MODEL_TASK_STREAM_LABEL_SWITCH_BY_INDEX);
domi::LabelSwitchByIndexDef *label_task_def = task_def1->mutable_label_switch_by_index();
label_task_def->set_op_index(op_index++);
label_task_def->set_label_max(2);
}

{
OpDescPtr op_desc = CreateOpDesc("label_then", LABELSET);
NodePtr node = graph->AddNode(op_desc); // op_index = 1
EXPECT_TRUE(AttrUtils::SetInt(op_desc, ATTR_NAME_LABEL_SWITCH_INDEX, 1));

domi::TaskDef *task_def1 = model_task_def->add_task();
task_def1->set_stream_id(0);
task_def1->set_type(RT_MODEL_TASK_LABEL_SET);
domi::LabelSetDef *label_task_def = task_def1->mutable_label_set();
label_task_def->set_op_index(op_index++);
}

{
OpDescPtr op_desc = CreateOpDesc("label_goto", LABELGOTOEX);
NodePtr node = graph->AddNode(op_desc); // op_index = 2
EXPECT_TRUE(AttrUtils::SetInt(op_desc, ATTR_NAME_LABEL_SWITCH_INDEX, 2));

domi::TaskDef *task_def2 = model_task_def->add_task();
task_def2->set_stream_id(0);
task_def2->set_type(RT_MODEL_TASK_STREAM_LABEL_GOTO);
domi::LabelGotoExDef *label_task_def = task_def2->mutable_label_goto_ex();
label_task_def->set_op_index(op_index++);
}


{
OpDescPtr op_desc = CreateOpDesc("label_else", LABELSET);
NodePtr node = graph->AddNode(op_desc); // op_index = 3
EXPECT_TRUE(AttrUtils::SetInt(op_desc, ATTR_NAME_LABEL_SWITCH_INDEX, 0));

domi::TaskDef *task_def1 = model_task_def->add_task();
task_def1->set_stream_id(0);
task_def1->set_type(RT_MODEL_TASK_LABEL_SET);
domi::LabelSetDef *label_task_def = task_def1->mutable_label_set();
label_task_def->set_op_index(op_index++);
}


{
OpDescPtr op_desc = CreateOpDesc("label_leave", LABELSET);
NodePtr node = graph->AddNode(op_desc); // op_index = 4
EXPECT_TRUE(AttrUtils::SetInt(op_desc, ATTR_NAME_LABEL_SWITCH_INDEX, 2));

domi::TaskDef *task_def1 = model_task_def->add_task();
task_def1->set_stream_id(0);
task_def1->set_type(RT_MODEL_TASK_LABEL_SET);
domi::LabelSetDef *label_task_def = task_def1->mutable_label_set();
label_task_def->set_op_index(op_index++);
}


EXPECT_TRUE(AttrUtils::SetInt(ge_model, ATTR_MODEL_LABEL_NUM, 3));
EXPECT_EQ(model.Assign(ge_model), SUCCESS);
EXPECT_EQ(model.Init(), SUCCESS);

EXPECT_EQ(model.input_addrs_list_.size(), 0);
EXPECT_EQ(model.output_addrs_list_.size(), 0);
EXPECT_EQ(model.task_list_.size(), 5);
}
} // namespace ge

+ 0
- 1
tests/ut/ge/single_op/single_op_manager_unittest.cc View File

@@ -17,7 +17,6 @@
#include <gtest/gtest.h>
#include <vector>

#include "cce/taskdown_common.hpp"
#include "runtime/rt.h"

#define protected public


Loading…
Cancel
Save