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.

hybrid_model_async_executor.cc 28 kB

4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623
  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/hybrid_model_async_executor.h"
  17. #include "graph/load/model_manager/model_utils.h"
  18. #include "graph/utils/tensor_utils.h"
  19. #include "graph/utils/type_utils.h"
  20. #include "graph/ge_context.h"
  21. #include "graph/types.h"
  22. #include "graph/debug/ge_attr_define.h"
  23. #include "graph/manager/graph_caching_allocator.h"
  24. #include "graph/manager/graph_mem_allocator.h"
  25. #include "graph/manager/rdma_pool_allocator.h"
  26. #include "graph/manager/host_mem_allocator.h"
  27. #include "graph/manager/graph_mem_manager.h"
  28. namespace ge {
  29. namespace hybrid {
  30. namespace {
  31. const int kDataOutputIndex = 0;
  32. const size_t kMinimumPiplineStages = 2;
  33. const int kDefaultLoopCount = 10;
  34. const size_t kAlignment = 64;
  35. }
  36. HybridModelAsyncExecutor::HybridModelAsyncExecutor(HybridModel *model)
  37. : model_(model), run_flag_(false), data_dumper_(nullptr) {
  38. }
  39. HybridModelAsyncExecutor::~HybridModelAsyncExecutor() {
  40. if (stream_ != nullptr) {
  41. GE_CHK_RT(rtStreamDestroy(stream_));
  42. }
  43. }
  44. void HybridModelAsyncExecutor::SetDeviceId(uint32_t device_id) {
  45. device_id_ = device_id;
  46. }
  47. void HybridModelAsyncExecutor::SetModelId(uint32_t model_id) {
  48. model_id_ = model_id;
  49. }
  50. Status HybridModelAsyncExecutor::EnqueueData(const shared_ptr<InputDataWrapper> &data) {
  51. if (data_inputer_->Push(data) != SUCCESS) {
  52. REPORT_CALL_ERROR("E19999", "Data queue is full, please call again later, model_id %u.", model_id_);
  53. GELOGE(domi::DATA_QUEUE_ISFULL,
  54. "[Push][Data] Data queue is full, please call again later, model_id %u ", model_id_);
  55. return domi::DATA_QUEUE_ISFULL;
  56. }
  57. GELOGD("EnqueueData successfully. model_id = %u, data_index = %u", data->GetInput().model_id, data->GetInput().index);
  58. return SUCCESS;
  59. }
  60. Status HybridModelAsyncExecutor::Start(const std::shared_ptr<ModelListener> &listener) {
  61. GELOGD("HybridModelExecutor::Start IN, has listener = %d", listener != nullptr);
  62. std::lock_guard<std::mutex> lk(mu_);
  63. if (run_flag_) {
  64. REPORT_INNER_ERROR("E19999", "Model already started, model_id:%u.", model_id_);
  65. GELOGE(INTERNAL_ERROR, "[Check][RunState] Model already started, model_id:%u.", model_id_);
  66. return INTERNAL_ERROR;
  67. }
  68. run_flag_ = true;
  69. listener_ = listener;
  70. future_ = std::async(std::launch::async, [&]() -> Status {
  71. GetThreadLocalContext() = *executor_->GetContext()->ge_context;
  72. GetContext().SetSessionId(executor_->GetContext()->session_id);
  73. GetContext().SetContextId(executor_->GetContext()->context_id);
  74. GE_CHECK_NOTNULL(executor_->GetContext()->ge_context);
  75. GetThreadLocalContext() = *executor_->GetContext()->ge_context;
  76. return RunInternal();
  77. });
  78. GE_CHK_BOOL_RET_STATUS(future_.valid(), INTERNAL_ERROR,
  79. "[Check][RunState] Failed to start, model_id:%u.", model_id_);
  80. GELOGD("HybridModelExecutor::Start successfully");
  81. return SUCCESS;
  82. }
  83. Status HybridModelAsyncExecutor::Stop() {
  84. std::lock_guard<std::mutex> lk(mu_);
  85. run_flag_ = false;
  86. data_inputer_->Stop();
  87. Status ret = SUCCESS;
  88. if (future_.valid()) {
  89. ret = future_.get();
  90. }
  91. if (is_op_debug_reg_) {
  92. op_debug_register_.UnregisterDebugForStream(stream_);
  93. }
  94. if (stream_ != nullptr) {
  95. GE_CHK_RT(rtStreamDestroy(stream_));
  96. stream_ = nullptr;
  97. }
  98. return ret;
  99. }
  100. Status HybridModelAsyncExecutor::Init() {
  101. data_inputer_ = std::unique_ptr<DataInputer>(new(std::nothrow) DataInputer());
  102. GE_CHECK_NOTNULL(data_inputer_);
  103. GE_CHK_RT_RET(rtStreamCreate(&stream_, RT_STREAM_PRIORITY_DEFAULT));
  104. executor_ = std::unique_ptr<HybridModelExecutor>(new(std::nothrow) HybridModelExecutor(model_, device_id_, stream_));
  105. GE_CHECK_NOTNULL(executor_);
  106. GE_CHK_STATUS_RET(executor_->Init(),
  107. "[Init][HybridModelExecutor] failed, model_id:%u.", model_id_);
  108. GE_CHK_STATUS_RET(DumpOpDebug(), "[Dump][OpDebug] failed, model_id:%u.", model_id_);
  109. GELOGI("HybridModel stage nums:%zu", model_->GetRootGraphItem()->NumGroups());
  110. if (model_->GetRootGraphItem()->NumGroups() >= kMinimumPiplineStages) {
  111. pipe_executor_ =
  112. std::unique_ptr<HybridModelPipelineExecutor>(new(std::nothrow) HybridModelPipelineExecutor(model_, device_id_));
  113. GE_CHECK_NOTNULL(pipe_executor_);
  114. GE_CHK_STATUS_RET(pipe_executor_->Init(),
  115. "[Init][HybridModelPipelineExecutor] failed, model_id:%u.", model_id_);
  116. }
  117. GE_CHK_STATUS_RET(InitInputDesc(), "[Init][InputDesc] failed, model_id:%u.", model_id_);
  118. return SUCCESS;
  119. }
  120. Status HybridModelAsyncExecutor::PreRun(InputData &current_data, HybridModelExecutor::ExecuteArgs &args) {
  121. GE_CHK_STATUS_RET(SyncVarData(), "[Invoke][SyncVarData] failed, model_id:%u.", model_id_);
  122. RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[SyncVarData] End");
  123. GE_CHK_STATUS_RET(PrepareInputs(current_data, args),
  124. "[Invoke][PrepareInputs] failed to copy input data to model, model_id:%u.", model_id_);
  125. RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[CopyInputData] End");
  126. return SUCCESS;
  127. }
  128. Status HybridModelAsyncExecutor::RunInternal() {
  129. auto device_id = static_cast<int32_t>(device_id_);
  130. GELOGD("Hybrid model start. model_id = %u, device_id = %u", model_id_, device_id_);
  131. GE_CHK_RT_RET(rtSetDevice(device_id));
  132. // DeviceReset before thread run finished!
  133. GE_MAKE_GUARD(not_used_var, [&] { GE_CHK_RT(rtDeviceReset(device_id)); });
  134. while (run_flag_) {
  135. // Model has not indeedly started running before received data
  136. SetRunningFlag(false);
  137. std::shared_ptr<InputDataWrapper> data_wrapper;
  138. Status ret = data_inputer_->Pop(data_wrapper);
  139. // Model indeedly start running
  140. SetRunningFlag(true);
  141. GE_IF_BOOL_EXEC(data_wrapper == nullptr || ret != SUCCESS, GELOGI("data_wrapper is null!, ret = %u", ret);
  142. continue);
  143. GELOGI("Getting the input data, model_id:%u", model_id_);
  144. GE_IF_BOOL_EXEC(!run_flag_, break);
  145. InputData current_data = data_wrapper->GetInput();
  146. GELOGI("Model thread Run begin, model id:%u, data index:%u.", model_id_, current_data.index);
  147. RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[RunInternal] [iteration = %d] Start", iterator_count_);
  148. HybridModelExecutor::ExecuteArgs args;
  149. ret = PreRun(current_data, args);
  150. GE_CHK_BOOL_TRUE_EXEC_WITH_LOG(
  151. ret != SUCCESS, (void) HandleResult(ret, current_data.index, args, data_wrapper->GetOutput());
  152. continue, "[Invoke][PreRun] failed, model_id:%u.", model_id_); // [No need to check value]
  153. if (pipe_executor_ != nullptr) {
  154. GELOGI("HybridModel will execute in pipeline mode");
  155. auto iter_per_run = std::getenv("ITER_NUM");
  156. if (iter_per_run) {
  157. args.num_loops = static_cast<int>(strtol(iter_per_run, nullptr, kDefaultLoopCount));
  158. }
  159. ret = pipe_executor_->Execute(args);
  160. } else {
  161. GELOGI("HybridModel will execute in singleline mode");
  162. ge::GetContext().SetSessionId(executor_->GetContext()->session_id);
  163. ge::GetContext().SetContextId(executor_->GetContext()->context_id);
  164. ret = executor_->Execute(args);
  165. }
  166. ret = HandleResult(ret, current_data.index, args, data_wrapper->GetOutput());
  167. if (ret != SUCCESS) {
  168. continue;
  169. }
  170. RECORD_MODEL_EXECUTION_EVENT(executor_->GetContext(), "[RunInternal] [iteration = %d] End", iterator_count_);
  171. iterator_count_++;
  172. SetRunningFlag(false);
  173. GELOGI("run iterator count is %lu, model_id:%u", iterator_count_, model_id_);
  174. }
  175. GELOGI("Model run end, model id:%u", model_id_);
  176. return SUCCESS;
  177. }
  178. Status HybridModelAsyncExecutor::HandleResult(Status exec_ret,
  179. uint32_t data_id,
  180. HybridModelExecutor::ExecuteArgs &args,
  181. OutputData *output_data) {
  182. GELOGD("Start to handle result. model id = %u, data index = %u, execution ret = %u", model_id_, data_id, exec_ret);
  183. std::vector<ge::Tensor> output_tensor_info_list;
  184. if (args.is_eos) {
  185. GELOGI("End of sequence, model id = %u", model_id_);
  186. GE_CHK_STATUS_RET_NOLOG(OnComputeDone(data_id, END_OF_SEQUENCE, output_tensor_info_list));
  187. return SUCCESS;
  188. }
  189. if (exec_ret != SUCCESS) {
  190. GELOGE(exec_ret, "[Check][Param:Status] failed to execute graph. model_id = %u", model_id_);
  191. REPORT_INNER_ERROR("E19999", "failed to execute graph. model_id = %u", model_id_);
  192. return OnComputeDone(data_id, INTERNAL_ERROR, output_tensor_info_list);
  193. }
  194. GE_CHECK_NOTNULL(output_data);
  195. auto ret = CopyOutputs(args, output_data, output_tensor_info_list);
  196. if (ret != SUCCESS) {
  197. OnComputeDone(data_id, INTERNAL_ERROR, output_tensor_info_list);
  198. return INTERNAL_ERROR;
  199. }
  200. GELOGD("Executed graph successfully, model id = %u, data_index = %u", model_id_, data_id);
  201. return OnComputeDone(data_id, SUCCESS, output_tensor_info_list);
  202. }
  203. Status HybridModelAsyncExecutor::SyncVarData() {
  204. GELOGI("Sync var data, model id:%u", model_id_);
  205. TensorValue *global_step_var = model_->GetVariable(NODE_NAME_GLOBAL_STEP);
  206. if (global_step_var != nullptr) {
  207. std::vector<uint64_t> v_step;
  208. v_step.push_back(iterator_count_);
  209. GE_CHK_RT_RET(rtMemcpy(global_step_var->MutableData(),
  210. global_step_var->GetSize(),
  211. v_step.data(),
  212. v_step.size() * sizeof(uint64_t),
  213. RT_MEMCPY_HOST_TO_DEVICE));
  214. } else {
  215. GELOGD("No GLOBAL_STEP variable was found.");
  216. }
  217. return SUCCESS;
  218. }
  219. Status HybridModelAsyncExecutor::PrepareInputs(const InputData &current_data, HybridModelExecutor::ExecuteArgs &args) {
  220. if (current_data.blobs.size() < input_tensor_desc_.size()) {
  221. GELOGE(PARAM_INVALID,
  222. "[Check][Size]Blob size mismatches, expect at least %zu, but got %zu, model_id = %u",
  223. input_tensor_desc_.size(), current_data.blobs.size(), model_id_);
  224. REPORT_INNER_ERROR("E19999", "Blob size mismatches, expect at least %zu, but got %zu, model_id = %u.",
  225. input_tensor_desc_.size(), current_data.blobs.size(), model_id_);
  226. return PARAM_INVALID;
  227. }
  228. auto allocator = NpuMemoryAllocator::GetAllocator(device_id_);
  229. GE_CHECK_NOTNULL(allocator);
  230. args.input_desc.resize(input_tensor_desc_.size());
  231. const std::vector<DataBuffer> &blobs = current_data.blobs;
  232. for (size_t input_index = 0; input_index < input_tensor_desc_.size(); ++input_index) {
  233. auto tensor_size = input_sizes_[input_index];
  234. if (is_input_dynamic_[input_index]) {
  235. if (input_index >= current_data.shapes.size()) {
  236. GELOGE(PARAM_INVALID,
  237. "[Check][Range]Shape index out of range, index = %zu, shape size = %zu model_id = %u.",
  238. input_index, current_data.shapes.size(), model_id_);
  239. REPORT_INNER_ERROR("E19999", "Shape index out of range, index = %zu, shape size = %zu, model_id = %u.",
  240. input_index, current_data.shapes.size(), model_id_);
  241. return PARAM_INVALID;
  242. }
  243. auto &tensor_desc = input_tensor_desc_[input_index];
  244. GeShape shape(current_data.shapes[input_index]);
  245. std::vector<std::pair<int64_t, int64_t>> range;
  246. auto range_ret = tensor_desc->GetShapeRange(range);
  247. GE_CHK_BOOL_RET_STATUS(range_ret == GRAPH_SUCCESS, INTERNAL_ERROR,
  248. "[Invoke][GetShapeRange] failed, ret=%u, model_id = %u.", range_ret, model_id_);
  249. for (size_t k = 0; k < range.size(); ++k) {
  250. if (k >= shape.GetDimNum()) {
  251. break;
  252. }
  253. // range[k].second can be -1
  254. if (shape.GetDim(k) < range[k].first || (range[k].second >= 0 && shape.GetDim(k) > range[k].second)) {
  255. GELOGE(PARAM_INVALID, "[Check][Range]Dim out of range, shape idx = %zu, dim idx = %zu,"
  256. "dim = %ld, range = [%ld, %ld], model_id = %u.",
  257. input_index, k, shape.GetDim(k), range[k].first, range[k].second, model_id_);
  258. REPORT_INNER_ERROR("E19999", "Dim out of range, shape idx = %zu, dim idx = %zu, dim = %ld,"
  259. "range = [%ld, %ld], model_id = %u.",
  260. input_index, k, shape.GetDim(k), range[k].first, range[k].second, model_id_);
  261. return PARAM_INVALID;
  262. }
  263. }
  264. tensor_desc->SetShape(shape);
  265. GELOGD("Update shape[%s] of input[%zu] to [%s]",
  266. shape.ToString().c_str(), input_index, tensor_desc->MutableShape().ToString().c_str());
  267. GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetTensorMemorySizeInBytes(*tensor_desc, tensor_size),
  268. "[Invoke][GetTensorMemorySizeInBytes]Failed to calc tensor size,"
  269. "index = %zu, shape = [%s], model_id = %u.",
  270. input_index, tensor_desc->GetShape().ToString().c_str(), model_id_);
  271. GELOGD("Input tensor[%zu] size = %ld", input_index, tensor_size);
  272. TensorUtils::SetSize(*tensor_desc, tensor_size);
  273. args.input_desc[input_index] = tensor_desc;
  274. }
  275. GE_CHECK_GE(tensor_size, 0);
  276. AllocationAttr attr;
  277. if (GetContext().GetHostExecFlag()) {
  278. attr.SetMemType(HOST_DDR);
  279. }
  280. auto tensor_buffer = TensorBuffer::Create(allocator, tensor_size, &attr);
  281. GE_CHECK_NOTNULL(tensor_buffer);
  282. args.inputs.emplace_back(std::shared_ptr<TensorBuffer>(tensor_buffer.release()));
  283. GELOGD("To copy input data for input[%zu]", input_index);
  284. const DataBuffer &data_buf = blobs[input_index];
  285. auto mem_size = static_cast<uint64_t>(tensor_size);
  286. if (mem_size < data_buf.length) {
  287. REPORT_INNER_ERROR("E19999",
  288. "input data size(%lu) does not match model required size(%lu), ret failed, model_id = %u.",
  289. data_buf.length, mem_size, model_id_);
  290. GELOGE(PARAM_INVALID,
  291. "[Check][Size]input data size(%lu) does not match model required size(%lu), ret failed, model_id = %u.",
  292. data_buf.length, mem_size, model_id_);
  293. return PARAM_INVALID;
  294. }
  295. if (data_buf.length > 0) {
  296. GELOGI("[IMAS]CopyPlainData memcpy graph_%u type[F] output[%zu] memaddr[%p] mem_size[%zu] datasize[%lu]",
  297. model_->root_runtime_param_.graph_id,
  298. input_index,
  299. args.inputs[input_index].GetData(),
  300. mem_size,
  301. data_buf.length);
  302. GE_CHK_RT_RET(rtMemcpy(args.inputs[input_index].MutableData(),
  303. mem_size,
  304. data_buf.data,
  305. data_buf.length,
  306. RT_MEMCPY_HOST_TO_DEVICE));
  307. }
  308. }
  309. return SUCCESS;
  310. }
  311. Status HybridModelAsyncExecutor::InitInputDesc() {
  312. int input_index = 0;
  313. for (const auto &input_node : model_->GetRootGraphItem()->GetInputNodes()) {
  314. GELOGD("Init input[%u], node = %s, is_dynamic = %d",
  315. input_index,
  316. input_node->NodeName().c_str(),
  317. input_node->is_dynamic);
  318. auto output_desc = input_node->MutableOutputDesc(kDataOutputIndex);
  319. GE_CHECK_NOTNULL(output_desc);
  320. int64_t tensor_size = -1;
  321. if (!input_node->is_dynamic) {
  322. GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetSize(*output_desc, tensor_size),
  323. "Failed to get size from %s",
  324. input_node->NodeName().c_str());
  325. if (tensor_size == 0) {
  326. GELOGW("[%s] Tensor size == 0", input_node->NodeName().c_str());
  327. GE_CHK_GRAPH_STATUS_RET(TensorUtils::GetTensorMemorySizeInBytes(*output_desc, tensor_size),
  328. "Failed to calc tensor size");
  329. GELOGD("[%s] Tensor size updated to %ld", input_node->NodeName().c_str(), tensor_size);
  330. }
  331. }
  332. input_sizes_.emplace(input_index, tensor_size);
  333. input_tensor_desc_.emplace(input_index, output_desc);
  334. is_input_dynamic_.push_back(input_node->is_dynamic);
  335. input_index += 1;
  336. }
  337. return SUCCESS;
  338. }
  339. Status HybridModelAsyncExecutor::OnComputeDone(uint32_t data_index, uint32_t result_code,
  340. std::vector<ge::Tensor> &outputs) {
  341. GELOGD("OnComputeDone. model id = %u, data index = %u, execution ret = %u", model_id_, data_index, result_code);
  342. if (listener_ != nullptr) {
  343. GE_CHK_STATUS(listener_->OnComputeDone(model_id_, data_index, result_code, outputs),
  344. "[Invoke][OnComputeDone] failed, model_id = %u.", model_id_);
  345. }
  346. return result_code;
  347. }
  348. Status HybridModelAsyncExecutor::CopyOutputs(HybridModelExecutor::ExecuteArgs &args, OutputData *output_data,
  349. std::vector<ge::Tensor> &outputs) {
  350. // copy output data from op to designated position
  351. std::vector<ConstGeTensorDescPtr> &output_tensor_desc_list = args.output_desc;
  352. std::vector<TensorValue> &output_tensors = args.outputs;
  353. if (output_tensor_desc_list.size() != output_tensors.size()) {
  354. GELOGE(INTERNAL_ERROR,
  355. "[Check][Size]Output sizes mismatch. From op_desc = %zu, and from output tensors = %zu, model_id = %u.",
  356. output_tensor_desc_list.size(), output_tensors.size(), model_id_);
  357. REPORT_INNER_ERROR("E19999",
  358. "Output sizes mismatch. From op_desc = %zu, and from output tensors = %zu, model_id = %u.",
  359. output_tensor_desc_list.size(), output_tensors.size(), model_id_);
  360. return INTERNAL_ERROR;
  361. }
  362. GELOGD("Number of outputs = %zu", output_tensor_desc_list.size());
  363. string execute_mode;
  364. auto result = ge::GetContext().GetOption(OPTION_EXEC_DYNAMIC_EXECUTE_MODE, execute_mode);
  365. if (result != SUCCESS) {
  366. GELOGW("Can not get dynamic execute mode attr");
  367. }
  368. GELOGD("The dynamic execute is %s", execute_mode.c_str());
  369. for (size_t i = 0; i < output_tensors.size(); ++i) {
  370. GELOGD("Start to process output[%zu]", i);
  371. auto &output_tensor = output_tensors[i];
  372. auto &tensor_desc = output_tensor_desc_list.at(i);
  373. GE_CHECK_NOTNULL(tensor_desc);
  374. int64_t output_size = -1;
  375. GE_CHK_GRAPH_STATUS_RET(TensorUtils::CalcTensorMemSize(tensor_desc->GetShape(),
  376. tensor_desc->GetFormat(),
  377. tensor_desc->GetDataType(),
  378. output_size),
  379. "[Calc][TensorMemSize]Failed for output[%zu]. shape = [%s], type = %s, format = %s",
  380. i,
  381. tensor_desc->GetShape().ToString().c_str(),
  382. TypeUtils::DataTypeToSerialString(tensor_desc->GetDataType()).c_str(),
  383. TypeUtils::FormatToSerialString(tensor_desc->GetFormat()).c_str());
  384. GELOGD("Got tensor size for output[%zu] successfully. shape = [%s], type = %s, format = %s, size = %ld",
  385. i,
  386. tensor_desc->GetShape().ToString().c_str(),
  387. TypeUtils::DataTypeToSerialString(tensor_desc->GetDataType()).c_str(),
  388. TypeUtils::FormatToSerialString(tensor_desc->GetFormat()).c_str(),
  389. output_size);
  390. GE_CHECK_GE(output_size, 0);
  391. GE_CHECK_LE(output_size, UINT32_MAX);
  392. if (output_tensor.GetSize() < static_cast<size_t>(output_size)) {
  393. GELOGE(INTERNAL_ERROR,
  394. "[Check][Size]output[%zu] tensor size(%zu) is not enough for output shape [%s], model_id = %u.",
  395. i, output_tensor.GetSize(), tensor_desc->GetShape().ToString().c_str(), model_id_);
  396. REPORT_INNER_ERROR("E19999", "output[%zu] tensor size(%zu) is not enough for output shape [%s] model_id = %u",
  397. i, output_tensor.GetSize(), tensor_desc->GetShape().ToString().c_str(), model_id_);
  398. return INTERNAL_ERROR;
  399. }
  400. GeShape ge_shape(tensor_desc->GetShape().GetDims());
  401. GeTensorDesc ge_tensor_desc;
  402. ge_tensor_desc.SetShape(ge_shape);
  403. if (output_size > 0) {
  404. if (execute_mode != kLazyRecompile) {
  405. auto aligned_ptr = MakeShared<AlignedPtr>(output_size, kAlignment);
  406. GE_CHECK_NOTNULL(aligned_ptr);
  407. auto data_buf = aligned_ptr->MutableGet();
  408. GE_CHECK_NOTNULL(data_buf);
  409. GE_CHK_RT_RET(rtMemcpy(data_buf, output_size, output_tensor.GetData(), output_size, RT_MEMCPY_DEVICE_TO_HOST));
  410. GeTensor ge_tensor(ge_tensor_desc);
  411. ge_tensor.SetData(aligned_ptr, output_size);
  412. output_data->blobs.emplace_back(data_buf, static_cast<uint32_t>(output_size), false);
  413. auto tensor = TensorAdapter::AsTensor(ge_tensor);
  414. outputs.emplace_back(std::move(tensor));
  415. } else {
  416. BuildDeviceTensor(output_tensor, ge_tensor_desc, output_size, outputs);
  417. output_data->blobs.emplace_back(output_tensor.Release(), static_cast<uint32_t>(output_size), false,
  418. static_cast<uint32_t>(kPlacementDevice));
  419. }
  420. } else {
  421. GELOGW("Output [%zu] is empty. shape = [%s]", i, tensor_desc->GetShape().ToString().c_str());
  422. GeTensor ge_tensor(ge_tensor_desc);
  423. ge_tensor.SetData(nullptr, 0U);
  424. output_data->blobs.emplace_back(nullptr, 0U, false);
  425. auto tensor = TensorAdapter::AsTensor(ge_tensor);
  426. outputs.emplace_back(std::move(tensor));
  427. }
  428. GELOGD("Output[%zu] added, type = %s, shape = [%s], size = %ld", i,
  429. TypeUtils::DataTypeToSerialString(tensor_desc->GetDataType()).c_str(),
  430. tensor_desc->GetShape().ToString().c_str(), output_size);
  431. }
  432. return SUCCESS;
  433. }
  434. void HybridModelAsyncExecutor::BuildDeviceTensor(TensorValue &output_tensor, GeTensorDesc &ge_tensor_desc,
  435. int64_t output_size, std::vector<ge::Tensor> &outputs) {
  436. GELOGD("Start to build device tensor");
  437. auto mem_type = output_tensor.GetMemType();
  438. GELOGD("Mem type is %d", static_cast<uint32_t>(mem_type));
  439. auto deleter = [=](uint8_t *device_data) {
  440. if (device_data != nullptr) {
  441. if (mem_type == RDMA_HBM) {
  442. MemManager::Instance().RdmaPoolInstance(RT_MEMORY_HBM).Free(device_data, device_id_);
  443. } else if (mem_type == HOST_DDR) {
  444. MemManager::Instance().HostMemInstance(RT_MEMORY_HBM).Free(device_data);
  445. } else {
  446. MemManager::Instance().CachingInstance(RT_MEMORY_HBM).Free(device_data, device_id_);
  447. }
  448. }
  449. };
  450. ge_tensor_desc.SetPlacement(kPlacementDevice);
  451. GeTensor ge_tensor(ge_tensor_desc);
  452. auto tensor = TensorAdapter::AsTensor(ge_tensor);
  453. tensor.SetData(reinterpret_cast<uint8_t *>(output_tensor.Release()), static_cast<size_t>(output_size), deleter);
  454. outputs.emplace_back(std::move(tensor));
  455. }
  456. Status HybridModelAsyncExecutor::Execute(const std::vector<DataBuffer> &inputs,
  457. const std::vector<GeTensorDesc> &input_desc,
  458. std::vector<DataBuffer> &outputs,
  459. std::vector<GeTensorDesc> &output_desc) {
  460. GELOGI("Start to execute model.");
  461. HybridModelExecutor::ExecuteArgs args;
  462. args.inputs.resize(inputs.size());
  463. for (size_t i = 0; i < inputs.size(); ++i) {
  464. TensorValue tensor_value(inputs[i].data, inputs[i].length);
  465. args.inputs[i] = tensor_value;
  466. }
  467. for (size_t i = 0; i < outputs.size(); ++i) {
  468. args.outputs.emplace_back(TensorValue(outputs[i].data, outputs[i].length));
  469. }
  470. // usr must designate input tensorDesc when input shape is dynamic in inference
  471. for (size_t i = 0; i < input_desc.size(); ++i) {
  472. ConstGeTensorDescPtr tensor_desc_ptr = MakeShared<GeTensorDesc>(input_desc[i]);
  473. args.input_desc.emplace_back(tensor_desc_ptr);
  474. }
  475. GE_CHK_STATUS_RET(executor_->Execute(args), "[Invoke][Execute] Failed, model_id = %u.", model_id_);
  476. for (const auto &output_tensor_desc : args.output_desc) {
  477. output_desc.emplace_back(*output_tensor_desc);
  478. }
  479. return SUCCESS;
  480. }
  481. Status HybridModelAsyncExecutor::Execute(const vector<GeTensor> &inputs, vector<GeTensor> &outputs) {
  482. GELOGD("Start to execute model.");
  483. // prepare inputs
  484. InputData input_data;
  485. for (auto &tensor : inputs) {
  486. DataBuffer buffer;
  487. buffer.data = const_cast<uint8_t *>(tensor.GetData().GetData());
  488. buffer.length = tensor.GetData().size();
  489. input_data.blobs.emplace_back(buffer);
  490. input_data.shapes.emplace_back(tensor.GetTensorDesc().GetShape().GetDims());
  491. }
  492. HybridModelExecutor::ExecuteArgs args;
  493. GE_CHK_STATUS_RET(PrepareInputs(input_data, args),
  494. "[Invoke][PrepareInputs]Failed to copy input data to model, model_id = %u", model_id_);
  495. GELOGD("Done copying input data successfully.");
  496. GE_CHK_STATUS_RET(executor_->Execute(args), "[Invoke][Execute] Failed, model_id = %u.", model_id_);
  497. std::vector<ge::Tensor> output_tensor_info_list;
  498. OutputData output_data;
  499. GE_CHK_STATUS_RET(CopyOutputs(args, &output_data, output_tensor_info_list),
  500. "[Invoke][CopyOutputs]Failed to copy outputs, model_id = %u.", model_id_);
  501. GELOGD("Done copying output data successfully. output count = %zu", output_tensor_info_list.size());
  502. int out_index = 0;
  503. outputs.resize(output_tensor_info_list.size());
  504. for (auto &out_tensor_info : output_tensor_info_list) {
  505. auto &ge_tensor = outputs[out_index];
  506. if (out_tensor_info.GetSize() > 0) {
  507. GE_CHK_GRAPH_STATUS_RET(ge_tensor.SetData(out_tensor_info.GetData(), out_tensor_info.GetSize()),
  508. "Failed to set output[%d].", out_index);
  509. }
  510. ge_tensor.MutableTensorDesc() = *args.output_desc[out_index];
  511. GELOGD("Set output[%d], tensor size = %ld, shape = [%s]",
  512. out_index,
  513. out_tensor_info.GetSize(),
  514. ge_tensor.MutableTensorDesc().MutableShape().ToString().c_str());
  515. ++out_index;
  516. }
  517. return SUCCESS;
  518. }
  519. Status HybridModelAsyncExecutor::DumpOpDebug() {
  520. const DumpProperties &dump_properties = executor_->GetContext()->dump_properties;
  521. if (dump_properties.IsOpDebugOpen()) {
  522. GELOGD("Opdebug is open in hybrid engine");
  523. uint32_t op_debug_mode = dump_properties.GetOpDebugMode();
  524. GE_CHK_RT_RET(op_debug_register_.RegisterDebugForStream(stream_, op_debug_mode, data_dumper_));
  525. is_op_debug_reg_ = true;
  526. data_dumper_.SetDumpProperties(dump_properties);
  527. data_dumper_.SetModelName(model_->GetModelName());
  528. data_dumper_.SetModelId(model_->GetModelId());
  529. data_dumper_.SetDeviceId(model_->GetDeviceId());
  530. void *global_step = nullptr;
  531. if (dump_properties.IsInferOpDebug()) {
  532. GELOGD("Init global step when infer with op debug.");
  533. global_step = executor_->GetContext()->global_step;
  534. } else {
  535. TensorValue *varible_global_step = model_->GetVariable(NODE_NAME_GLOBAL_STEP);
  536. if (varible_global_step != nullptr) {
  537. global_step = const_cast<void *>(varible_global_step->GetData());
  538. }
  539. }
  540. void *loop_per_iter = nullptr;
  541. TensorValue *varible_loop_per_iter = model_->GetVariable(NODE_NAME_FLOWCTRL_LOOP_PER_ITER);
  542. if (varible_loop_per_iter != nullptr) {
  543. loop_per_iter = const_cast<void *>(varible_loop_per_iter->GetData());
  544. }
  545. void *loop_cond = nullptr;
  546. TensorValue *varible_loop_cond = model_->GetVariable(NODE_NAME_FLOWCTRL_LOOP_COND);
  547. if (varible_loop_cond != nullptr) {
  548. loop_cond = const_cast<void *>(varible_loop_cond->GetData());
  549. }
  550. data_dumper_.SetLoopAddr(global_step, loop_per_iter, loop_cond);
  551. GE_CHK_STATUS_RET(data_dumper_.LoadDumpInfo(),
  552. "[Invoke][LoadDumpInfo] failed in hybrid engine, model_id = %u.", model_id_);
  553. GELOGD("Dump op debug SUCCESS in hybrid engine");
  554. }
  555. return SUCCESS;
  556. }
  557. } // namespace hybrid
  558. } // namespace ge

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