You can not select more than 25 topics Topics must start with a chinese character,a letter or number, can include dashes ('-') and can be up to 35 characters long.

subgraph_executor.cc 23 kB

5 years ago
5 years ago
5 years ago
4 years ago
5 years ago
5 years ago
5 years ago
5 years ago
4 years ago
5 years ago
4 years ago
5 years ago
5 years ago
5 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
5 years ago
4 years ago
4 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
5 years ago
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508
  1. /**
  2. * Copyright 2019-2020 Huawei Technologies Co., Ltd
  3. *
  4. * Licensed under the Apache License, Version 2.0 (the "License");
  5. * you may not use this file except in compliance with the License.
  6. * You may obtain a copy of the License at
  7. *
  8. * http://www.apache.org/licenses/LICENSE-2.0
  9. *
  10. * Unless required by applicable law or agreed to in writing, software
  11. * distributed under the License is distributed on an "AS IS" BASIS,
  12. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  13. * See the License for the specific language governing permissions and
  14. * limitations under the License.
  15. */
  16. #include "hybrid/executor/subgraph_executor.h"
  17. #include "graph/ge_context.h"
  18. #include "hybrid/executor/worker/task_compile_engine.h"
  19. #include "hybrid/executor/worker/execution_engine.h"
  20. #include "hybrid/node_executor/node_executor.h"
  21. namespace ge {
  22. namespace hybrid {
  23. namespace {
  24. constexpr int kDefaultThreadNum = 4;
  25. constexpr int kDefaultQueueSize = 16;
  26. constexpr int kDataInputIndex = 0;
  27. }
  28. SubgraphExecutor::SubgraphExecutor(const GraphItem *graph_item, GraphExecutionContext *context, bool force_infer_shape)
  29. : graph_item_(graph_item),
  30. context_(context),
  31. force_infer_shape_(force_infer_shape),
  32. pre_run_pool_(kDefaultThreadNum),
  33. ready_queue_(kDefaultQueueSize) {
  34. }
  35. SubgraphExecutor::~SubgraphExecutor() {
  36. GELOGD("[%s] SubgraphExecutor destroyed.", graph_item_->GetName().c_str());
  37. }
  38. Status SubgraphExecutor::Init(const std::vector<TensorValue> &inputs,
  39. const std::vector<ConstGeTensorDescPtr> &input_desc) {
  40. subgraph_context_.reset(new(std::nothrow)SubgraphContext(graph_item_, context_));
  41. GE_CHECK_NOTNULL(subgraph_context_);
  42. GE_CHK_STATUS_RET(subgraph_context_->Init(),
  43. "[Init][SubgraphContext][%s] Failed to init subgraph context.", graph_item_->GetName().c_str());
  44. shape_inference_engine_.reset(new(std::nothrow) ShapeInferenceEngine(context_, subgraph_context_.get()));
  45. GE_CHECK_NOTNULL(shape_inference_engine_);
  46. if (graph_item_->IsDynamic()) {
  47. GE_CHK_STATUS_RET(InitInputsForUnknownShape(inputs, input_desc),
  48. "[%s] Failed to set inputs.",
  49. graph_item_->GetName().c_str());
  50. } else {
  51. GE_CHK_STATUS_RET(InitInputsForKnownShape(inputs),
  52. "[Invoke][InitInputsForKnownShape][%s] Failed to init subgraph executor for known shape subgraph.",
  53. graph_item_->GetName().c_str());
  54. }
  55. return SUCCESS;
  56. }
  57. Status SubgraphExecutor::InitInputsForUnknownShape(const std::vector<TensorValue> &inputs,
  58. const std::vector<ConstGeTensorDescPtr> &input_desc) {
  59. // Number of inputs of parent node should be greater or equal than that of subgraph
  60. auto input_nodes = graph_item_->GetInputNodes();
  61. if (inputs.size() < input_nodes.size()) {
  62. GELOGE(INTERNAL_ERROR,
  63. "[Check][Size][%s] Number of inputs [%zu] is not sufficient for subgraph which needs [%zu] inputs.",
  64. graph_item_->GetName().c_str(), inputs.size(), input_nodes.size());
  65. REPORT_INNER_ERROR("E19999",
  66. "[%s] Number of inputs [%zu] is not sufficient for subgraph which needs [%zu] inputs,"
  67. "check invalid when SubgraphExecutor %s.",
  68. graph_item_->GetName().c_str(), inputs.size(), input_nodes.size(), __FUNCTION__);
  69. return INTERNAL_ERROR;
  70. }
  71. for (size_t i = 0; i < input_nodes.size(); ++i) {
  72. auto &input_node = input_nodes[i];
  73. if (input_node == nullptr) {
  74. GELOGD("[%s] Input[%zu] is not needed by subgraph, skip it.", graph_item_->GetName().c_str(), i);
  75. continue;
  76. }
  77. auto &input_tensor = inputs[i];
  78. GELOGD("[%s] Set input tensor[%zu] to inputs with index = %d, tensor = %s",
  79. graph_item_->GetName().c_str(),
  80. i,
  81. input_node->input_start,
  82. input_tensor.DebugString().c_str());
  83. GE_CHK_STATUS_RET(subgraph_context_->SetInput(*input_node, kDataInputIndex, input_tensor),
  84. "[Invoke][SetInput] failed for grap_item[%s] input tensor[%zu]", graph_item_->GetName().c_str(), i);
  85. if (force_infer_shape_ || input_node->is_dynamic) {
  86. GELOGD("[%s] Start to update input[%zu] for subgraph data node.", graph_item_->GetName().c_str(), i);
  87. GE_CHECK_LE(i + 1, input_desc.size());
  88. const auto &tensor_desc = input_desc[i];
  89. GE_CHECK_NOTNULL(tensor_desc);
  90. auto node_state = subgraph_context_->GetOrCreateNodeState(input_node);
  91. GE_CHECK_NOTNULL(node_state);
  92. node_state->GetShapeInferenceState().UpdateInputShape(0, *tensor_desc);
  93. }
  94. }
  95. GELOGD("[%s] Done setting inputs.", graph_item_->GetName().c_str());
  96. return SUCCESS;
  97. }
  98. Status SubgraphExecutor::InitInputsForKnownShape(const std::vector<TensorValue> &inputs) {
  99. auto &input_index_mapping = graph_item_->GetInputIndexMapping();
  100. for (size_t i = 0; i < input_index_mapping.size(); ++i) {
  101. auto &parent_input_index = input_index_mapping[i];
  102. if (static_cast<size_t>(parent_input_index) >= inputs.size()) {
  103. GELOGE(INTERNAL_ERROR,
  104. "[Check][Size][%s] Number of inputs [%zu] is not sufficient for subgraph which needs at lease [%d] inputs",
  105. graph_item_->GetName().c_str(), inputs.size(), parent_input_index + 1);
  106. REPORT_INNER_ERROR("E19999",
  107. "[%s] Number of inputs [%zu] is not sufficient for subgraph which needs at lease [%d] inputs,"
  108. "check invalid when %s.",
  109. graph_item_->GetName().c_str(), inputs.size(), parent_input_index + 1, __FUNCTION__);
  110. return INTERNAL_ERROR;
  111. }
  112. auto &input_tensor = inputs[parent_input_index];
  113. subgraph_context_->SetInput(static_cast<int>(i), input_tensor);
  114. GELOGD("[%s] Set input tensor[%zu] with inputs with index = %d, tensor = %s",
  115. graph_item_->GetName().c_str(),
  116. i,
  117. parent_input_index,
  118. input_tensor.DebugString().c_str());
  119. }
  120. return SUCCESS;
  121. }
  122. Status SubgraphExecutor::ExecuteAsync(const std::vector<TensorValue> &inputs,
  123. const std::vector<ConstGeTensorDescPtr> &input_desc,
  124. const std::vector<TensorValue> &outputs) {
  125. GELOGD("[%s] is dynamic = %s", graph_item_->GetName().c_str(), graph_item_->IsDynamic() ? "true" : "false");
  126. GE_CHK_STATUS_RET(Init(inputs, input_desc), "[Invoke][Init]failed for [%s].", graph_item_->GetName().c_str());
  127. if (!outputs.empty()) {
  128. GE_CHK_STATUS_RET(EnableOutputZeroCopy(outputs),
  129. "[Invoke][EnableOutputZeroCopy] Failed by user provided outputs.");
  130. }
  131. if (!graph_item_->IsDynamic()) {
  132. return ExecuteAsyncForKnownShape(inputs);
  133. }
  134. HYBRID_CHK_STATUS_RET(ScheduleTasks(), "[%s] Failed to execute tasks.", graph_item_->GetName().c_str());
  135. GELOGD("[%s] Done executing subgraph successfully.", graph_item_->GetName().c_str());
  136. return SUCCESS;
  137. }
  138. Status SubgraphExecutor::ExecuteAsync(const std::vector<TensorValue> &inputs,
  139. const std::vector<ConstGeTensorDescPtr> &input_desc) {
  140. return ExecuteAsync(inputs, input_desc, {});
  141. }
  142. Status SubgraphExecutor::ExecuteAsyncForKnownShape(const std::vector<TensorValue> &inputs) {
  143. GELOGD("[%s] subgraph is not dynamic.", graph_item_->GetName().c_str());
  144. if (graph_item_->GetAllNodes().size() != 1) {
  145. GELOGE(INTERNAL_ERROR,
  146. "[%s] Invalid known shape subgraph. node size = %zu",
  147. graph_item_->GetName().c_str(),
  148. graph_item_->GetAllNodes().size());
  149. return INTERNAL_ERROR;
  150. }
  151. auto node_item = graph_item_->GetAllNodes()[0];
  152. GE_CHECK_NOTNULL(node_item);
  153. auto node_state = subgraph_context_->GetOrCreateNodeState(node_item);
  154. GE_CHECK_NOTNULL(node_state);
  155. node_state->SetKernelTask(node_item->kernel_task);
  156. known_shape_task_context_ = TaskContext::Create(node_state.get(), context_, subgraph_context_.get());
  157. GE_CHECK_NOTNULL(known_shape_task_context_);
  158. HYBRID_CHK_STATUS_RET(ExecutionEngine::ExecuteAsync(*node_state, known_shape_task_context_, *context_),
  159. "[%s] Failed to execute node [%s] for known subgraph.",
  160. graph_item_->GetName().c_str(),
  161. known_shape_task_context_->GetNodeName());
  162. GELOGD("[%s] Done execute non-dynamic subgraph successfully.", graph_item_->GetName().c_str());
  163. return SUCCESS;
  164. }
  165. Status SubgraphExecutor::ExecuteAsync(TaskContext &task_context) {
  166. std::vector<TensorValue> inputs;
  167. std::vector<ConstGeTensorDescPtr> input_desc;
  168. for (int i = 0; i < task_context.NumInputs(); ++i) {
  169. auto tensor = task_context.GetInput(i);
  170. GE_CHECK_NOTNULL(tensor);
  171. inputs.emplace_back(*tensor);
  172. input_desc.emplace_back(task_context.GetInputDesc(i));
  173. }
  174. GE_CHK_STATUS_RET(ExecuteAsync(inputs, input_desc),
  175. "[Invoke][ExecuteAsync] failed for [%s].", graph_item_->GetName().c_str());
  176. GE_CHK_STATUS_RET(SetOutputsToParentNode(task_context),
  177. "[Invoke][SetOutputsToParentNode][%s] Failed to set output shapes to parent node.",
  178. graph_item_->GetName().c_str());
  179. return SUCCESS;
  180. }
  181. Status SubgraphExecutor::PrepareNodes(int group) {
  182. GELOGD("[%s] Start to prepare nodes. group = %d",
  183. graph_item_->GetName().c_str(),
  184. group);
  185. auto &all_nodes = graph_item_->GetAllNodes(group);
  186. for (auto all_node : all_nodes) {
  187. auto &node_item = *all_node;
  188. // for while op
  189. if (force_infer_shape_ && !node_item.is_dynamic) {
  190. GELOGD("[%s] Force infer shape is set, updating node to dynamic.", node_item.NodeName().c_str());
  191. auto &mutable_node_item = const_cast<NodeItem &>(node_item);
  192. mutable_node_item.SetToDynamic();
  193. }
  194. GELOGD("[%s] Start to prepare node [%s].", graph_item_->GetName().c_str(), node_item.NodeName().c_str());
  195. auto node_state = subgraph_context_->GetOrCreateNodeState(&node_item);
  196. GE_CHECK_NOTNULL(node_state);
  197. auto p_node_state = node_state.get();
  198. if (node_item.node_type != NETOUTPUT) {
  199. // only do shape inference and compilation for nodes with dynamic shapes.
  200. if (node_item.is_dynamic) {
  201. auto prepare_future = pre_run_pool_.commit([this, p_node_state]() -> Status {
  202. GetContext().SetSessionId(context_->session_id);
  203. GetContext().SetContextId(context_->context_id);
  204. GE_CHK_STATUS_RET_NOLOG(InferShape(shape_inference_engine_.get(), *p_node_state));
  205. return PrepareForExecution(context_, *p_node_state);
  206. });
  207. p_node_state->SetPrepareFuture(std::move(prepare_future));
  208. } else {
  209. GELOGD("[%s] Skipping shape inference and compilation for node with static shape.",
  210. node_item.NodeName().c_str());
  211. if (node_item.kernel_task == nullptr) {
  212. GELOGW("[%s] Node of static shape got no task.", node_item.NodeName().c_str());
  213. GE_CHK_STATUS_RET(TaskCompileEngine::Compile(*p_node_state, context_),
  214. "[Invoke][Compile] failed for [%s].", p_node_state->GetName().c_str());
  215. } else {
  216. node_state->SetKernelTask(node_item.kernel_task);
  217. }
  218. auto unique_task_context =
  219. TaskContext::Create(node_state.get(), context_, subgraph_context_.get());
  220. GE_CHECK_NOTNULL(unique_task_context);
  221. const auto &task = node_state->GetKernelTask();
  222. if (task == nullptr) {
  223. GELOGE(INTERNAL_ERROR, "[Get][KernelTask] failed for[%s], NodeTask is null.", node_state->GetName().c_str());
  224. REPORT_CALL_ERROR("E19999", "invoke GetKernelTask failed for %s when %s, nodetask is null.",
  225. node_state->GetName().c_str(), __FUNCTION__);
  226. return INTERNAL_ERROR;
  227. }
  228. auto shared_task_context = std::shared_ptr<TaskContext>(unique_task_context.release());
  229. node_state->SetTaskContext(shared_task_context);
  230. }
  231. }
  232. if (!ready_queue_.Push(p_node_state)) {
  233. if (context_->is_eos_) {
  234. GELOGD("Got end of sequence");
  235. return SUCCESS;
  236. }
  237. GELOGE(INTERNAL_ERROR, "[Check][State][%s] Error occurs while launching tasks. quit from preparing nodes.",
  238. graph_item_->GetName().c_str());
  239. REPORT_INNER_ERROR("E19999", "[%s] Error occurs while launching tasks. quit from preparing nodes when %s.",
  240. graph_item_->GetName().c_str(), __FUNCTION__);
  241. return INTERNAL_ERROR;
  242. }
  243. GELOGD("[%s] Push node [%s] to queue.", graph_item_->GetName().c_str(), node_item.NodeName().c_str());
  244. }
  245. GELOGD("[%s] Done preparing nodes successfully.", graph_item_->GetName().c_str());
  246. return SUCCESS;
  247. }
  248. Status SubgraphExecutor::InferShape(ShapeInferenceEngine *shape_inference_engine, NodeState &node_state) const {
  249. HYBRID_CHK_STATUS_RET(shape_inference_engine->InferShape(node_state),
  250. "[Invoke][InferShape] failed for [%s].", node_state.GetName().c_str());
  251. HYBRID_CHK_STATUS_RET(shape_inference_engine->PropagateOutputShapes(node_state),
  252. "[Invoke][PropagateOutputShapes] failed for [%s].", node_state.GetName().c_str());
  253. return SUCCESS;
  254. }
  255. Status SubgraphExecutor::PrepareForExecution(GraphExecutionContext *ctx, NodeState &node_state) {
  256. auto &node_item = *node_state.GetNodeItem();
  257. if (node_item.kernel_task == nullptr) {
  258. GE_CHK_STATUS_RET(TaskCompileEngine::Compile(node_state, ctx),
  259. "[Invoke][Compile] Failed for node[%s]", node_state.GetName().c_str());
  260. } else {
  261. node_state.SetKernelTask(node_item.kernel_task);
  262. }
  263. auto unique_task_context = TaskContext::Create(&node_state, context_, subgraph_context_.get());
  264. GE_CHECK_NOTNULL(unique_task_context);
  265. const auto &task = node_state.GetKernelTask();
  266. if (task == nullptr) {
  267. GELOGE(INTERNAL_ERROR, "[Invoke][GetKernelTask] failed for[%s], NodeTask is null.", node_state.GetName().c_str());
  268. REPORT_CALL_ERROR("E19999", "invoke GetKernelTask failed for %s, NodeTask is null when %s.",
  269. node_state.GetName().c_str(), __FUNCTION__);
  270. return INTERNAL_ERROR;
  271. }
  272. auto shared_task_context = std::shared_ptr<TaskContext>(unique_task_context.release());
  273. node_state.SetTaskContext(shared_task_context);
  274. GE_CHK_RT_RET(rtCtxSetCurrent(ctx->rt_context));
  275. RECORD_COMPILE_EVENT(ctx, node_item.NodeName().c_str(), "[UpdateTilingData] start");
  276. GE_CHK_STATUS_RET_NOLOG(task->UpdateTilingData(*shared_task_context)); // update op_desc before alloc ws
  277. RECORD_COMPILE_EVENT(ctx, node_item.NodeName().c_str(), "[UpdateTilingData] end");
  278. return SUCCESS;
  279. }
  280. Status SubgraphExecutor::LaunchTasks() {
  281. while (true) {
  282. NodeState *node_state = nullptr;
  283. if (!ready_queue_.Pop(node_state)) {
  284. GELOGE(INTERNAL_ERROR, "[Invoke][Pop] failed for [%s].", graph_item_->GetName().c_str());
  285. REPORT_CALL_ERROR("E19999", "invoke pop failed for %s when %s", graph_item_->GetName().c_str(), __FUNCTION__);
  286. return INTERNAL_ERROR;
  287. }
  288. if (node_state == nullptr) {
  289. GELOGD("[%s] Got EOF from queue.", graph_item_->GetName().c_str());
  290. return SUCCESS;
  291. }
  292. if (node_state->GetType() == NETOUTPUT) {
  293. // Wait for all inputs become valid
  294. // after PrepareNodes returned. all output tensors and shapes are valid
  295. GE_CHK_STATUS_RET_NOLOG(node_state->GetShapeInferenceState().AwaitShapesReady(*context_));
  296. GE_CHK_STATUS_RET_NOLOG(node_state->AwaitInputTensors(*context_));
  297. GELOGD("[%s] Done executing node successfully.", node_state->GetName().c_str());
  298. continue;
  299. }
  300. GE_CHK_STATUS_RET_NOLOG(node_state->WaitForPrepareDone());
  301. GELOGD("[%s] Start to execute.", node_state->GetName().c_str());
  302. auto shared_task_context = node_state->GetTaskContext();
  303. GE_CHECK_NOTNULL(shared_task_context);
  304. shared_task_context->SetForceInferShape(force_infer_shape_);
  305. HYBRID_CHK_STATUS_RET(ExecutionEngine::ExecuteAsync(*node_state, shared_task_context, *context_),
  306. "[Invoke][ExecuteAsync] failed for [%s].", node_state->GetName().c_str());
  307. GELOGD("[%s] Done executing node successfully.", node_state->GetName().c_str());
  308. }
  309. }
  310. Status SubgraphExecutor::ScheduleTasks(int group) {
  311. GELOGD("[%s] Start to schedule prepare workers.", graph_item_->GetName().c_str());
  312. auto prepare_future = std::async(std::launch::async, [&]() -> Status {
  313. GetContext().SetSessionId(context_->session_id);
  314. GetContext().SetContextId(context_->context_id);
  315. auto ret = PrepareNodes(group);
  316. ready_queue_.Push(nullptr);
  317. return ret;
  318. });
  319. GELOGD("[%s] Start to execute subgraph.", graph_item_->GetName().c_str());
  320. auto ret = LaunchTasks();
  321. if (ret != SUCCESS) {
  322. subgraph_context_->OnError(ret);
  323. context_->SetErrorCode(ret);
  324. ready_queue_.Stop();
  325. prepare_future.wait();
  326. return ret;
  327. }
  328. GE_CHK_STATUS_RET(prepare_future.get(),
  329. "[Invoke][get] [%s] Error occurred in task preparation.", graph_item_->GetName().c_str());
  330. GELOGD("[%s] Done launching all tasks successfully.", graph_item_->GetName().c_str());
  331. return SUCCESS;
  332. }
  333. Status SubgraphExecutor::GetOutputs(vector<TensorValue> &outputs) {
  334. return subgraph_context_->GetOutputs(outputs);
  335. }
  336. Status SubgraphExecutor::GetOutputs(vector<TensorValue> &outputs, std::vector<ConstGeTensorDescPtr> &output_desc) {
  337. GE_CHK_STATUS_RET(GetOutputs(outputs), "[Invoke][GetOutputs] failed for [%s].", graph_item_->GetName().c_str());
  338. // copy output data from op to designated position
  339. GE_CHK_STATUS_RET(graph_item_->GetOutputDescList(output_desc),
  340. "[Invoke][GetOutputDescList][%s] Failed to get output tensor desc.", graph_item_->GetName().c_str());
  341. if (outputs.size() != output_desc.size()) {
  342. GELOGE(INTERNAL_ERROR,
  343. "[Check][Size]Number of outputs(%zu) mismatch number of output_desc(%zu).",
  344. outputs.size(), output_desc.size());
  345. REPORT_INNER_ERROR("E19999", "Number of outputs(%zu) mismatch number of output_desc(%zu),"
  346. "check invlid when SubgraphExecutor %s.",
  347. outputs.size(), output_desc.size(), __FUNCTION__);
  348. return INTERNAL_ERROR;
  349. }
  350. return SUCCESS;
  351. }
  352. Status SubgraphExecutor::Synchronize() {
  353. GELOGD("[%s] Synchronize start.", graph_item_->GetName().c_str());
  354. GE_CHK_STATUS_RET_NOLOG(context_->Synchronize(context_->stream));
  355. GELOGD("[%s] Done synchronizing successfully.", graph_item_->GetName().c_str());
  356. return SUCCESS;
  357. }
  358. Status SubgraphExecutor::SetOutputsToParentNode(TaskContext &task_context) {
  359. // get output tensors and tensor desc list
  360. std::vector<TensorValue> outputs;
  361. std::vector<ConstGeTensorDescPtr> output_desc_list;
  362. GE_CHK_STATUS_RET(subgraph_context_->GetOutputs(outputs),
  363. "[Invoke][GetOutputs][%s] Failed to get output tensors.", graph_item_->GetName().c_str());
  364. GE_CHK_STATUS_RET(graph_item_->GetOutputDescList(output_desc_list),
  365. "[Invoke][GetOutputDescList][%s] Failed to get output tensor desc.", graph_item_->GetName().c_str());
  366. if (outputs.size() != output_desc_list.size()) {
  367. GELOGE(INTERNAL_ERROR, "[Check][Size][%s] num of output tensors = %zu, num of output tensor desc = %zu not equal",
  368. graph_item_->GetName().c_str(), outputs.size(), output_desc_list.size());
  369. REPORT_INNER_ERROR("E19999",
  370. "%s num of output tensors = %zu, num of output tensor desc = %zu not equal,"
  371. "check invalid when SubgraphExecutor %s",
  372. graph_item_->GetName().c_str(), outputs.size(), output_desc_list.size(), __FUNCTION__);
  373. return INTERNAL_ERROR;
  374. }
  375. // mapping to parent task context
  376. for (size_t i = 0; i < outputs.size(); ++i) {
  377. int parent_output_index = graph_item_->GetParentOutputIndex(i);
  378. GE_CHECK_GE(parent_output_index, 0);
  379. // update tensor
  380. GELOGD("[%s] Updating output[%zu] to parent output[%d]",
  381. graph_item_->GetName().c_str(),
  382. i,
  383. parent_output_index);
  384. GELOGD("[%s] Updating output tensor, index = %d, tensor = %s",
  385. graph_item_->GetName().c_str(),
  386. parent_output_index,
  387. outputs[i].DebugString().c_str());
  388. GE_CHK_STATUS_RET(task_context.SetOutput(parent_output_index, outputs[i]));
  389. // updating shapes. dynamic format/dtype is not supported.
  390. // It should be noted that even the subgraph is of known shape, it is also necessary to update parent output desc,
  391. // for instance, IfOp may have two known-shaped subgraphs of different output shapes
  392. const auto &output_desc = output_desc_list[i];
  393. auto parent_output_desc = task_context.MutableOutputDesc(parent_output_index);
  394. GE_CHECK_NOTNULL(parent_output_desc);
  395. GELOGD("[%s] Updating output shape[%d] from [%s] to [%s]",
  396. graph_item_->GetName().c_str(),
  397. parent_output_index,
  398. parent_output_desc->MutableShape().ToString().c_str(),
  399. output_desc->GetShape().ToString().c_str());
  400. parent_output_desc->SetShape(output_desc->GetShape());
  401. GELOGD("[%s] Updating output original shape[%d] from [%s] to [%s]",
  402. graph_item_->GetName().c_str(),
  403. parent_output_index,
  404. parent_output_desc->GetOriginShape().ToString().c_str(),
  405. output_desc->GetOriginShape().ToString().c_str());
  406. parent_output_desc->SetOriginShape(output_desc->GetOriginShape());
  407. }
  408. return SUCCESS;
  409. }
  410. Status SubgraphExecutor::EnableOutputZeroCopy(const vector<TensorValue> &outputs) {
  411. GELOGD("To enable zero copy, output number = %zu", outputs.size());
  412. const auto &output_edges = graph_item_->GetOutputEdges();
  413. // Op -> MetOutput, set the output tensor of Op that output to the NetOutput node
  414. if (outputs.size() != output_edges.size()) {
  415. GELOGE(PARAM_INVALID, "[Check][Size]Output number mismatches, expect = %zu, but given = %zu",
  416. output_edges.size(), outputs.size());
  417. REPORT_INNER_ERROR("E19999", "Output number mismatches, expect = %zu, but given = %zu when %s",
  418. output_edges.size(), outputs.size(), __FUNCTION__);
  419. return PARAM_INVALID;
  420. }
  421. for (size_t i = 0; i < outputs.size(); ++i) {
  422. auto &output_tensor = outputs[i];
  423. auto &output_node = output_edges[i].first;
  424. int output_idx = output_edges[i].second;
  425. GELOGD("[%s] Set output tensor[%zu] to [%s]'s output[%d], tensor = %s",
  426. graph_item_->GetName().c_str(),
  427. i,
  428. output_node->NodeName().c_str(),
  429. output_idx,
  430. output_tensor.DebugString().c_str());
  431. GE_CHK_STATUS_RET(subgraph_context_->SetOutput(*output_node, output_idx, output_tensor),
  432. "[Invoke][SetOutput][%s] Failed to set input tensor[%zu]", graph_item_->GetName().c_str(), i);
  433. }
  434. GELOGD("Done enabling zero copy for outputs successfully.");
  435. return SUCCESS;
  436. }
  437. Status SubgraphExecutor::PartialExecuteAsync(int task_group) {
  438. return ScheduleTasks(task_group);
  439. }
  440. Status SubgraphExecutor::InitForPartialExecution(const vector<TensorValue> &inputs,
  441. const vector<ConstGeTensorDescPtr> &input_desc) {
  442. return Init(inputs, input_desc);
  443. }
  444. } // namespace hybrid
  445. } // namespace ge

图引擎模块(GE)是MindSpore的一个子模块,其代码由C++实现,位于前端模块ME和底层硬件之间,起到承接作用。图引擎模块以ME下发的图作为输入,然后进行一系列的深度图优化操作,最后输出一张可以在底层硬件上高效运行的图。GE针对昇腾AI处理器的硬件结构特点,做了特定的优化工作,以此来充分发挥出昇腾AI处理器的强大算力。在进行模型训练/推理时,GE会被自动调用而用户并不感知。GE主要由GE API和GE Core两部分组成,详细的架构图如下所示