diff --git a/.gitignore b/.gitignore index 8a4003cf..891c0f87 100644 --- a/.gitignore +++ b/.gitignore @@ -2,6 +2,7 @@ /build /output /prebuilts +/cov *.ir *.out diff --git a/ge/CMakeLists.txt b/ge/CMakeLists.txt index 16494a33..be47c8dd 100755 --- a/ge/CMakeLists.txt +++ b/ge/CMakeLists.txt @@ -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" diff --git a/ge/graph/build/run_context.cc b/ge/graph/build/run_context.cc index 50094cf3..ba328840 100644 --- a/ge/graph/build/run_context.cc +++ b/ge/graph/build/run_context.cc @@ -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(rt_ret), i); return RT_FAILED; diff --git a/ge/graph/load/model_manager/davinci_model.cc b/ge/graph/load/model_manager/davinci_model.cc index ae0d421a..ed2428d9 100755 --- a/ge/graph/load/model_manager/davinci_model.cc +++ b/ge/graph/load/model_manager/davinci_model.cc @@ -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; diff --git a/ge/graph/passes/multi_batch_clone_pass.cc b/ge/graph/passes/multi_batch_clone_pass.cc index 17a1e3bb..b8fb6bde 100755 --- a/ge/graph/passes/multi_batch_clone_pass.cc +++ b/ge/graph/passes/multi_batch_clone_pass.cc @@ -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 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; diff --git a/ge/graph/passes/multi_batch_clone_pass.h b/ge/graph/passes/multi_batch_clone_pass.h index 66e92892..0dae88ca 100755 --- a/ge/graph/passes/multi_batch_clone_pass.h +++ b/ge/graph/passes/multi_batch_clone_pass.h @@ -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> batch_shapes_; diff --git a/ge/graph/preprocess/multi_batch_copy_graph.cc b/ge/graph/preprocess/multi_batch_copy_graph.cc index e43c5dd2..215b31ee 100644 --- a/ge/graph/preprocess/multi_batch_copy_graph.cc +++ b/ge/graph/preprocess/multi_batch_copy_graph.cc @@ -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"); diff --git a/ge/graph/preprocess/multi_batch_options.cc b/ge/graph/preprocess/multi_batch_options.cc index 84f38fa6..3bde0efb 100644 --- a/ge/graph/preprocess/multi_batch_options.cc +++ b/ge/graph/preprocess/multi_batch_options.cc @@ -377,7 +377,7 @@ bool InitDynamicParams(vector> &shapes) { /// @param [out] map>> &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> &shapes, +Status ParserDataToDynamicInfo(const vector> &shapes, vector>> &data_name_and_shape, map> > &data_to_dynamic_info) { size_t cur_data_index = 0; diff --git a/ge/graph/preprocess/multi_batch_options.h b/ge/graph/preprocess/multi_batch_options.h index 9baf4f43..bfe96ea2 100644 --- a/ge/graph/preprocess/multi_batch_options.h +++ b/ge/graph/preprocess/multi_batch_options.h @@ -74,7 +74,7 @@ Status CalcShape(const std::vector &batch_shape, GeShape &data_shape); /// @param [out] map>> &data_to_dynamic_info: key:data_name. value:dynamic dims. /// @return SUCCESS / PARAM_INVALID /// -Status ParserDataToDynmaicInfo(const vector> &shapes, +Status ParserDataToDynamicInfo(const vector> &shapes, vector>> &data_name_and_shape, map>> &data_to_dynamic_info); diff --git a/ge/hybrid/common/tensor_value.cc b/ge/hybrid/common/tensor_value.cc index 16ecfaa4..c691c6f3 100644 --- a/ge/hybrid/common/tensor_value.cc +++ b/ge/hybrid/common/tensor_value.cc @@ -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(); } diff --git a/ge/hybrid/executor/hybrid_model_executor.cc b/ge/hybrid/executor/hybrid_model_executor.cc index c47dafc1..9c4bb217 100755 --- a/ge/hybrid/executor/hybrid_model_executor.cc +++ b/ge/hybrid/executor/hybrid_model_executor.cc @@ -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; diff --git a/ge/hybrid/executor/subgraph_executor.cc b/ge/hybrid/executor/subgraph_executor.cc index f8f122b1..8b194233 100644 --- a/ge/hybrid/executor/subgraph_executor.cc +++ b/ge/hybrid/executor/subgraph_executor.cc @@ -131,10 +131,14 @@ Status SubgraphExecutor::InitInputsForKnownShape(const std::vector } Status SubgraphExecutor::ExecuteAsync(const std::vector &inputs, - const std::vector &input_desc) { + const std::vector &input_desc, + const std::vector &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 &inputs, return SUCCESS; } +Status SubgraphExecutor::ExecuteAsync(const std::vector &inputs, + const std::vector &input_desc) { + return ExecuteAsync(inputs, input_desc, {}); +} + Status SubgraphExecutor::ExecuteAsyncForKnownShape(const std::vector &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 &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 diff --git a/ge/hybrid/executor/subgraph_executor.h b/ge/hybrid/executor/subgraph_executor.h index 4523e2c4..2b7e9371 100644 --- a/ge/hybrid/executor/subgraph_executor.h +++ b/ge/hybrid/executor/subgraph_executor.h @@ -43,7 +43,19 @@ class SubgraphExecutor { * @param input_desc input tensor descriptions * @return SUCCESS on success, error code otherwise */ - Status ExecuteAsync(const std::vector &inputs, const std::vector &input_desc); + Status ExecuteAsync(const std::vector &inputs, + const std::vector &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 &inputs, + const std::vector &input_desc, + const std::vector &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 &outputs); static Status InferShape(ShapeInferenceEngine *shape_inference_engine, NodeState &node_state); Status Init(const std::vector &inputs, const std::vector &input_desc); diff --git a/ge/hybrid/model/hybrid_model.cc b/ge/hybrid/model/hybrid_model.cc index 7e5d8fe5..4511c2b9 100644 --- a/ge/hybrid/model/hybrid_model.cc +++ b/ge/hybrid/model/hybrid_model.cc @@ -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; } diff --git a/ge/hybrid/model/hybrid_model.h b/ge/hybrid/model/hybrid_model.h index 72495cad..1f973d1e 100644 --- a/ge/hybrid/model/hybrid_model.h +++ b/ge/hybrid/model/hybrid_model.h @@ -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> 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 weight_buffer_; RuntimeParam root_runtime_param_; }; } // namespace hybrid diff --git a/ge/hybrid/model/hybrid_model_builder.cc b/ge/hybrid/model/hybrid_model_builder.cc index b314c6a7..03e76bc9 100755 --- a/ge/hybrid/model/hybrid_model_builder.cc +++ b/ge/hybrid/model/hybrid_model_builder.cc @@ -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(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(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 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 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(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(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) { diff --git a/ge/hybrid/model/hybrid_model_builder.h b/ge/hybrid/model/hybrid_model_builder.h index 045bf3ef..71663a6e 100644 --- a/ge/hybrid/model/hybrid_model_builder.h +++ b/ge/hybrid/model/hybrid_model_builder.h @@ -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 &dependencies); Status ParseDependentForFusedSubgraph(NodeItem &node_item); Status IndexTaskDefs(); + Status IndexTaskDefs(const ComputeGraphPtr &sub_graph, const GeModelPtr &ge_model); Status IndexSpecialNodes(); Status InitRuntimeParams(); Status InitModelMem(); diff --git a/ge/hybrid/node_executor/aicore/aicore_node_executor.cc b/ge/hybrid/node_executor/aicore/aicore_node_executor.cc index cb5a7d4c..3174df80 100755 --- a/ge/hybrid/node_executor/aicore/aicore_node_executor.cc +++ b/ge/hybrid/node_executor/aicore/aicore_node_executor.cc @@ -49,6 +49,7 @@ Status AiCoreNodeExecutor::Initialize() { Status AiCoreNodeExecutor::LoadTask(const HybridModel &model, const NodePtr &node, shared_ptr &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 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; diff --git a/ge/hybrid/node_executor/aicore/aicore_op_task.cc b/ge/hybrid/node_executor/aicore/aicore_op_task.cc index f1bd6466..a34bba22 100644 --- a/ge/hybrid/node_executor/aicore/aicore_op_task.cc +++ b/ge/hybrid/node_executor/aicore/aicore_op_task.cc @@ -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()); diff --git a/ge/hybrid/node_executor/aicore/aicore_op_task.h b/ge/hybrid/node_executor/aicore/aicore_op_task.h index 3f350531..69a74ea9 100755 --- a/ge/hybrid/node_executor/aicore/aicore_op_task.h +++ b/ge/hybrid/node_executor/aicore/aicore_op_task.h @@ -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 output_indices_to_skip_; }; diff --git a/ge/hybrid/node_executor/aicore/aicore_task_builder.cc b/ge/hybrid/node_executor/aicore/aicore_task_builder.cc index b2996435..2bf2cb36 100755 --- a/ge/hybrid/node_executor/aicore/aicore_task_builder.cc +++ b/ge/hybrid/node_executor/aicore/aicore_task_builder.cc @@ -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 &node_task, bool ignore_failure_on_atomic) { +Status AiCoreTaskBuilder::BuildTask(std::unique_ptr &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 &node_task, bool i auto atomic_task = std::unique_ptr(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 &node_task, bool i // build aicore task auto aicore_task = std::unique_ptr(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()); diff --git a/ge/hybrid/node_executor/aicore/aicore_task_builder.h b/ge/hybrid/node_executor/aicore/aicore_task_builder.h index 92db809d..8f95df15 100755 --- a/ge/hybrid/node_executor/aicore/aicore_task_builder.h +++ b/ge/hybrid/node_executor/aicore/aicore_task_builder.h @@ -47,7 +47,7 @@ class AiCoreTaskBuilder { AiCoreTaskBuilder(const OpDescPtr &op_desc, const std::vector &task_defs); ~AiCoreTaskBuilder() = default; - Status BuildTask(std::unique_ptr &node_task, bool ignore_failure_on_atomic); + Status BuildTask(std::unique_ptr &node_task, bool ignore_failure_on_atomic, bool is_single_op = false); private: bool ExpectAtomicAddrCleanTask(); diff --git a/ge/single_op/single_op.cc b/ge/single_op/single_op.cc index 4f32bd6b..168ca2c5 100755 --- a/ge/single_op/single_op.cc +++ b/ge/single_op/single_op.cc @@ -256,9 +256,27 @@ Status DynamicSingleOp::ExecuteAsync(const vector &input_desc, const vector &input_buffers, vector &output_desc, vector &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(tensor_desc); + GE_CHECK_NOTNULL(desc); + args.input_desc.emplace_back(desc); + } + + return hybrid_model_executor_->Execute(args); + } + std::lock_guard 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)); diff --git a/ge/single_op/single_op.h b/ge/single_op/single_op.h index d677f94a..b350b684 100755 --- a/ge/single_op/single_op.h +++ b/ge/single_op/single_op.h @@ -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 &inputs, const std::vector &outputs); friend class SingleOpModel; - StreamResource *stream_resource_; + StreamResource *stream_resource_ = nullptr; std::mutex *stream_mutex_; rtStream_t stream_ = nullptr; std::vector input_addr_list_; @@ -77,6 +78,8 @@ class DynamicSingleOp { std::vector &outputs) const; std::unique_ptr op_task_; + std::unique_ptr hybrid_model_; + std::unique_ptr hybrid_model_executor_; uintptr_t resource_id_ = 0; std::mutex *stream_mutex_; rtStream_t stream_ = nullptr; diff --git a/ge/single_op/single_op_model.cc b/ge/single_op/single_op_model.cc index 7d092091..1b776cc8 100755 --- a/ge/single_op/single_op_model.cc +++ b/ge/single_op/single_op_model.cc @@ -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 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 diff --git a/ge/single_op/stream_resource.cc b/ge/single_op/stream_resource.cc index db6b7c47..a3acf6b7 100755 --- a/ge/single_op/stream_resource.cc +++ b/ge/single_op/stream_resource.cc @@ -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; } diff --git a/ge/single_op/stream_resource.h b/ge/single_op/stream_resource.h index d5bc941a..d2c1ca36 100755 --- a/ge/single_op/stream_resource.h +++ b/ge/single_op/stream_resource.h @@ -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); diff --git a/tests/depends/runtime/src/runtime_stub.cc b/tests/depends/runtime/src/runtime_stub.cc index 9b45e7e2..1a170167 100644 --- a/tests/depends/runtime/src/runtime_stub.cc +++ b/tests/depends/runtime/src/runtime_stub.cc @@ -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(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; diff --git a/tests/ut/ge/CMakeLists.txt b/tests/ut/ge/CMakeLists.txt index f21319d4..384acc30 100755 --- a/tests/ut/ge/CMakeLists.txt +++ b/tests/ut/ge/CMakeLists.txt @@ -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 diff --git a/tests/ut/ge/graph/load/davinci_model_unittest.cc b/tests/ut/ge/graph/load/davinci_model_unittest.cc index b6a5a180..0a6532fb 100644 --- a/tests/ut/ge/graph/load/davinci_model_unittest.cc +++ b/tests/ut/ge/graph/load/davinci_model_unittest.cc @@ -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(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("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("default"); + + GeModelPtr ge_model = make_shared(); + 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 model_task_def = make_shared(); + 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 diff --git a/tests/ut/ge/single_op/single_op_manager_unittest.cc b/tests/ut/ge/single_op/single_op_manager_unittest.cc index a70d2984..05da8683 100644 --- a/tests/ut/ge/single_op/single_op_manager_unittest.cc +++ b/tests/ut/ge/single_op/single_op_manager_unittest.cc @@ -17,7 +17,6 @@ #include #include -#include "cce/taskdown_common.hpp" #include "runtime/rt.h" #define protected public