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.

dnnengine_manager.cc 23 kB

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
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
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
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
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
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
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
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
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545
  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 "engine_manager/dnnengine_manager.h"
  17. #include <unistd.h>
  18. #include <cstdio>
  19. #include <fstream>
  20. #include <map>
  21. #include <utility>
  22. #include "common/debug/log.h"
  23. #include "common/ge/ge_util.h"
  24. #include "common/util/error_manager/error_manager.h"
  25. #include "framework/common/debug/ge_log.h"
  26. #include "analyzer/analyzer.h"
  27. #include "graph/ge_context.h"
  28. #include "graph/utils/graph_utils.h"
  29. #include "graph/utils/node_utils.h"
  30. #include "init/gelib.h"
  31. #include "framework/common/types.h"
  32. namespace {
  33. const char *const kSchedulerUnits = "schedule_units";
  34. const char *const kId = "id";
  35. const char *const kName = "name";
  36. const char *const kExAttrs = "ex_attrs";
  37. const char *const kIndependent = "independent";
  38. const char *const kSkipAssignStream = "skip_assign_stream";
  39. const char *const kCalEngines = "cal_engines";
  40. const char *const kAttch = "attach";
  41. const char *const kVectorCore = "VectorCore";
  42. const char *const kVectorEngine = "VectorEngine";
  43. const char *const kAIcoreEngine = "AIcoreEngine";
  44. const char *const kCustomOpFlag = "_custom_op_flag";
  45. const char *const kHostCpuEngineName = "DNN_VM_HOST_CPU";
  46. const char *const kHostCpuOpKernelLibName = "DNN_VM_HOST_CPU_OP_STORE";
  47. } // namespace
  48. namespace ge {
  49. namespace {
  50. const std::set<std::string> kNotCpuOp = {DATA, CONSTANT, CONSTANTOP, VARIABLE, NETOUTPUT};
  51. bool ExecOnHostCpu(const OpDescPtr &op_desc) {
  52. bool is_host_cpu_op = (kNotCpuOp.find(op_desc->GetType()) == kNotCpuOp.end());
  53. return ge::GetContext().GetHostExecFlag() && is_host_cpu_op;
  54. }
  55. } // namespace
  56. DNNEngineManager::DNNEngineManager() : init_flag_(false) {}
  57. DNNEngineManager::~DNNEngineManager() {
  58. engines_attrs_map_.clear();
  59. schedulers_.clear();
  60. }
  61. Status DNNEngineManager::Initialize(const std::map<std::string, std::string> &options) {
  62. // Multiple initializations are not supported
  63. if (init_flag_) {
  64. GELOGW("DNNEngineManager has been initialized.");
  65. return SUCCESS;
  66. }
  67. // Load engine so
  68. std::string so_path = "plugin/nnengine/";
  69. std::string path = PluginManager::GetPath();
  70. path.append(so_path);
  71. std::string so_api_func = "GetDNNEngineObjs";
  72. std::vector<std::string> so_func{so_api_func};
  73. Status status = plugin_mgr_.Load(path, so_func);
  74. if (status != SUCCESS) {
  75. GELOGE(status, "[Load][EngineSo]Failed, lib path %s", path.c_str());
  76. REPORT_CALL_ERROR("E19999", "Load engine so failed, lib path %s", path.c_str());
  77. return status;
  78. }
  79. status = plugin_mgr_.InvokeAll<std::map<std::string, DNNEnginePtr> &>(so_api_func, engines_map_);
  80. if (status != SUCCESS) {
  81. GELOGE(status, "[Get][DNNEngineObjs]Failed, so_api_func %s", so_api_func.c_str());
  82. REPORT_CALL_ERROR("E19999", "Get DNNEngineObjs failed, so_api_func %s", so_api_func.c_str());
  83. return status;
  84. }
  85. GELOGI("The number of DNNEngineObjs is %zu.", engines_map_.size());
  86. // Engines initialize
  87. for (auto iter = engines_map_.begin(); iter != engines_map_.end(); ++iter) {
  88. if (iter->second == nullptr) {
  89. GELOGI("Engine: %s point to nullptr", (iter->first).c_str());
  90. continue;
  91. }
  92. GELOGI("DNNEngine name: %s.", (iter->first).c_str());
  93. status = iter->second->Initialize(options);
  94. if (status != SUCCESS) {
  95. GELOGE(status, "[Init][Engine]Failed, engine %s", (iter->first).c_str());
  96. REPORT_CALL_ERROR("E19999", "Initialize engine %s failed", (iter->first).c_str());
  97. return status;
  98. }
  99. // Check engines' attribute
  100. DNNEngineAttribute attrs;
  101. iter->second->GetAttributes(attrs);
  102. if (attrs.runtime_type == RuntimeType::DEVICE) {
  103. if ((attrs.mem_type.size()) != 1 || (attrs.mem_type[0] != GE_ENGINE_ATTR_MEM_TYPE_HBM)) {
  104. GELOGE(GE_ENG_MEMTYPE_ERROR, "[Check][Param]Engine %s in aicore, but the memory type is "
  105. "not HBM, mem_type_size %lu", (iter->first).c_str(), attrs.mem_type.size());
  106. REPORT_INNER_ERROR("E19999", "Engine %s in aicore, but the memory type is not HBM, "
  107. "mem_type_size %lu", (iter->first).c_str(), attrs.mem_type.size());
  108. return GE_ENG_MEMTYPE_ERROR;
  109. }
  110. }
  111. }
  112. status = ParserJsonFile();
  113. if (status != SUCCESS) {
  114. GELOGE(status, "[Parse][JsonFile]Failed");
  115. return status;
  116. }
  117. status = CheckJsonFile();
  118. if (status != SUCCESS) {
  119. GELOGE(status, "[Check][JsonFile]Failed");
  120. return status;
  121. }
  122. init_flag_ = true;
  123. return SUCCESS;
  124. }
  125. Status DNNEngineManager::Finalize() {
  126. // Finalize is not allowed, initialize first is necessary
  127. if (!init_flag_) {
  128. GELOGW("DNNEngineManager has been finalized.");
  129. return SUCCESS;
  130. }
  131. for (auto iter = engines_map_.begin(); iter != engines_map_.end(); ++iter) {
  132. if (iter->second != nullptr) {
  133. GELOGI("DNNEngine name: %s.", (iter->first).c_str());
  134. Status status = iter->second->Finalize();
  135. if (status != SUCCESS) {
  136. GELOGE(status, "[Finalize][Engine]Failed, engine %s", (iter->first).c_str());
  137. REPORT_CALL_ERROR("E19999", "Finalize engine %s failed", (iter->first).c_str());
  138. return status;
  139. }
  140. }
  141. }
  142. init_flag_ = false;
  143. engines_map_.clear();
  144. return SUCCESS;
  145. }
  146. std::shared_ptr<ge::DNNEngine> DNNEngineManager::GetEngine(const std::string &name) const {
  147. auto iter = engines_map_.find(name);
  148. if (iter != engines_map_.end()) {
  149. return iter->second;
  150. }
  151. GELOGW("Failed to get engine object by engine name. %s.", name.c_str());
  152. return nullptr;
  153. }
  154. bool DNNEngineManager::IsEngineRegistered(const std::string &name) {
  155. auto iter = engines_map_.find(name);
  156. if (iter != engines_map_.end()) {
  157. return true;
  158. }
  159. GELOGW("Engine: %s is not Registered", name.c_str());
  160. return false;
  161. }
  162. void DNNEngineManager::InitPerformanceStaistic() {
  163. std::lock_guard<std::mutex> lock(mutex_);
  164. checksupport_cost_.clear();
  165. }
  166. const map<string, uint64_t> &DNNEngineManager::GetCheckSupportCost() const {
  167. std::lock_guard<std::mutex> lock(mutex_);
  168. return checksupport_cost_;
  169. }
  170. std::string DNNEngineManager::GetDNNEngineName(const ge::NodePtr &node_ptr) {
  171. std::lock_guard<std::mutex> lock(mutex_);
  172. GE_IF_BOOL_EXEC(node_ptr == nullptr, GELOGE(GE_CLI_GE_NOT_INITIALIZED, "DNNEngineManager: node_ptr is nullptr");
  173. return "");
  174. auto op_desc = node_ptr->GetOpDesc();
  175. GE_IF_BOOL_EXEC(op_desc == nullptr, GELOGE(GE_CLI_GE_NOT_INITIALIZED, "DNNEngineManager: op_desc is nullptr");
  176. return "");
  177. // Use the OpsKernelManager in GELib to get the opInfos for this opCode
  178. std::shared_ptr<GELib> instance_ptr = ge::GELib::GetInstance();
  179. if ((instance_ptr == nullptr) || (!instance_ptr->InitFlag())) {
  180. GELOGE(GE_CLI_GE_NOT_INITIALIZED, "[Get][DNNEngineName]Failed, gelib not init before");
  181. REPORT_INNER_ERROR("E19999", "Get DNNEngineName failed, gelib not init before");
  182. return "";
  183. }
  184. OpsKernelManager &ops_kernel_manager = instance_ptr->OpsKernelManagerObj();
  185. std::vector<OpInfo> op_infos = ops_kernel_manager.GetOpsKernelInfo(op_desc->GetType());
  186. if (op_infos.empty()) {
  187. GELOGI("DNNEngineManager: Can not get op info by op type %s", op_desc->GetType().c_str());
  188. return "";
  189. }
  190. GE_IF_BOOL_EXEC(ExecOnHostCpu(op_desc), return GetHostCpuEngineName(op_infos, op_desc));
  191. std::string ge_core_type;
  192. Status ret = ge::GetContext().GetOption(ge::CORE_TYPE, ge_core_type);
  193. GE_IF_BOOL_EXEC(ret != SUCCESS, GELOGD("get the option CORE_TYPE fail, set it to default value VECTOR_ENGINE"));
  194. std::string exclude_core_Type = (ge_core_type == kVectorCore) ? kAIcoreEngine : kVectorEngine;
  195. GELOGD("engine type will exclude: %s", exclude_core_Type.c_str());
  196. auto root_graph = ge::GraphUtils::FindRootGraph(node_ptr->GetOwnerComputeGraph());
  197. std::map<std::string, std::string> unsupported_reasons;
  198. for (const auto &it : op_infos) {
  199. if (it.engine == exclude_core_Type) {
  200. continue;
  201. }
  202. auto &kernel_map = ops_kernel_manager.GetAllOpsKernelInfoStores();
  203. auto &kernel_name = it.opKernelLib;
  204. auto kernel_info_store = kernel_map.find(kernel_name);
  205. if (kernel_info_store != kernel_map.end()) {
  206. std::string unsupported_reason;
  207. // It will be replaced by engine' checksupport
  208. uint64_t start_time = GetCurrentTimestamp();
  209. if (kernel_info_store->second->CheckSupported(node_ptr, unsupported_reason)) {
  210. checksupport_cost_[kernel_name] += GetCurrentTimestamp() - start_time;
  211. op_desc->SetOpEngineName(it.engine);
  212. op_desc->SetOpKernelLibName(kernel_name);
  213. // set attrs for taking information when load txt to graph object
  214. (void) AttrUtils::SetStr(op_desc, ATTR_NAME_ENGINE_NAME_FOR_LX, it.engine);
  215. (void) AttrUtils::SetStr(op_desc, ATTR_NAME_KKERNEL_LIB_NAME_FOR_LX, kernel_name);
  216. GELOGD("DNNEngineManager:Set OpKernelLibName %s and engine name %s to op_desc %s", kernel_name.c_str(),
  217. it.engine.c_str(), op_desc->GetName().c_str());
  218. return it.engine;
  219. } else {
  220. checksupport_cost_[kernel_name] += GetCurrentTimestamp() - start_time;
  221. bool is_custom_op = false;
  222. if ((ge::AttrUtils::GetBool(op_desc, kCustomOpFlag, is_custom_op)) && is_custom_op) {
  223. ErrorManager::GetInstance().ATCReportErrMessage("E13001", {"kernelname", "optype", "opname"},
  224. {kernel_name, op_desc->GetType(), op_desc->GetName()});
  225. GELOGE(FAILED,
  226. "[Check][Param]The custom operator registered by the user does not support "
  227. "the logic function delivered by this network, kernel_name %s, op type %s, "
  228. "op name %s",
  229. kernel_name.c_str(), op_desc->GetType().c_str(), op_desc->GetName().c_str());
  230. std::string error_info = "The custom operator registered by the user does not support the logic function"
  231. "delivered by this network";
  232. return "";
  233. }
  234. unsupported_reasons.emplace(kernel_name, unsupported_reason);
  235. GELOGI("DNNEngineManager:Check support failed, kernel_name is %s, op type is %s, op name is %s",
  236. kernel_name.c_str(), op_desc->GetType().c_str(), op_desc->GetName().c_str());
  237. if (!op_desc->HasAttr("_is_ge_op")) {
  238. ErrorManager::GetInstance().ATCReportErrMessage("W11001", {"opname"}, {op_desc->GetName()});
  239. }
  240. }
  241. } else {
  242. GELOGW(
  243. "DNNEngineManager:Can not find any supported ops kernel info store by kernel_name %s,"
  244. "op type is %s, op name is %s",
  245. kernel_name.c_str(), op_desc->GetType().c_str(), op_desc->GetName().c_str());
  246. }
  247. }
  248. // concat unsupported reasons analyzed data selection
  249. string reason;
  250. for (const auto &it : unsupported_reasons) {
  251. reason += it.first + ":" + it.second + ";";
  252. ErrorManager::GetInstance().ATCReportErrMessage(
  253. "E13002", {"optype", "opskernel", "reason"}, {op_desc->GetType(), it.first, it.second});
  254. GELOGE(GE_GRAPH_ASSIGN_ENGINE_FAILED, "[Check][OpSupported]Op type %s of ops kernel %s "
  255. "is unsupported, reason %s",
  256. op_desc->GetType().c_str(), it.first.c_str(), it.second.c_str());
  257. }
  258. analyzer::DataInfo analyze_info{root_graph->GetSessionID(), root_graph->GetGraphID(),
  259. analyzer::CHECKSUPPORT, node_ptr, reason};
  260. // do not change original process
  261. (void)Analyzer::GetInstance()->DoAnalyze(analyze_info);
  262. ErrorManager::GetInstance().ATCReportErrMessage(
  263. "E13003", {"opname", "optype"}, {op_desc->GetName(), op_desc->GetType()});
  264. GELOGE(GE_GRAPH_ASSIGN_ENGINE_FAILED, "[Get][DNNEngineName]Can't find any supported ops kernel "
  265. "and engine of %s, type is %s",
  266. op_desc->GetName().c_str(), op_desc->GetType().c_str());
  267. return "";
  268. }
  269. std::string DNNEngineManager::GetHostCpuEngineName(const std::vector<OpInfo> &op_infos,
  270. const OpDescPtr &op_desc) const {
  271. for (const auto &it : op_infos) {
  272. if ((it.engine == kHostCpuEngineName) && (it.opKernelLib == kHostCpuOpKernelLibName)) {
  273. op_desc->SetOpEngineName(kHostCpuEngineName);
  274. op_desc->SetOpKernelLibName(kHostCpuOpKernelLibName);
  275. GELOGI("DNNEngineManager: Set OpKernelLibName %s and OpEngineName %s to %s",
  276. kHostCpuOpKernelLibName, kHostCpuEngineName, op_desc->GetName().c_str());
  277. return kHostCpuEngineName;
  278. }
  279. }
  280. GELOGE(FAILED, "[Get][HostCpuEngineName]Failed, HostCpuEngine not support [%s, %s]",
  281. op_desc->GetName().c_str(), op_desc->GetType().c_str());
  282. REPORT_INNER_ERROR("E19999", "Get HostCpuEngineName failed, HostCpuEngine not support [%s, %s]",
  283. op_desc->GetName().c_str(), op_desc->GetType().c_str());
  284. return "";
  285. }
  286. const std::map<std::string, SchedulerConf> &DNNEngineManager::GetSchedulers() const { return schedulers_; }
  287. Status DNNEngineManager::ParserJsonFile() {
  288. GELOGI("Begin to parser json file");
  289. std::string json_file_path = "plugin/nnengine/ge_config/engine_conf.json";
  290. std::string path = PluginManager::GetPath();
  291. path.append(json_file_path);
  292. nlohmann::json scheduler_json_file;
  293. Status status = ReadJsonFile(path, &scheduler_json_file);
  294. if (status != SUCCESS) {
  295. GELOGE(FAILED, "[Read][JsonFile]Failed, file %s", path.c_str());
  296. REPORT_CALL_ERROR("E19999", "Read json file %s failed", path.c_str());
  297. return FAILED;
  298. }
  299. if (scheduler_json_file.is_null()) {
  300. // when engine_conf.json is not exist, just return success
  301. GELOGW("Json file is null");
  302. return SUCCESS;
  303. }
  304. try {
  305. nlohmann::json scheduler_utils_json = scheduler_json_file[kSchedulerUnits];
  306. if (scheduler_utils_json.is_null()) {
  307. GELOGE(FAILED, "[Check[Param]Find scheduler units failed, the message is null, file %s", path.c_str());
  308. REPORT_INNER_ERROR("E19999", "Find scheduler units failed, the message is null, file %s", path.c_str());
  309. return FAILED;
  310. }
  311. if (!scheduler_utils_json.is_array()) {
  312. GELOGE(FAILED, "[Check][Param]The message of kSchedulerUnits is not array and "
  313. "the file path is %s", path.c_str());
  314. REPORT_INNER_ERROR("E19999", "The message of kSchedulerUnits is not array and "
  315. "the file path is %s", path.c_str());
  316. return FAILED;
  317. }
  318. auto size = scheduler_json_file[kSchedulerUnits].size();
  319. for (size_t i = 0; i < size; i++) {
  320. SchedulerConf scheduler_conf;
  321. std::map<std::string, EngineConfPtr> engine_conf_map;
  322. nlohmann::json engines_json_map = scheduler_utils_json[i][kCalEngines];
  323. if (engines_json_map.is_null()) {
  324. GELOGE(FAILED, "[Check][Param]The message of cal_engines is null, file %s", path.c_str());
  325. REPORT_INNER_ERROR("E19999", "The message of cal_engines is null, file %s", path.c_str());
  326. return FAILED;
  327. }
  328. std::string scheduler_id_temp = scheduler_utils_json[i][kId];
  329. if (!scheduler_id_temp.empty()) {
  330. scheduler_conf.id = scheduler_id_temp;
  331. } else {
  332. GELOGE(FAILED, "[Check][Param]Scheduler ID is null, file %s", path.c_str());
  333. REPORT_INNER_ERROR("E19999", "Scheduler ID is null, file %s", path.c_str());
  334. return FAILED;
  335. }
  336. status = ParserEngineMessage(engines_json_map, scheduler_id_temp, engine_conf_map);
  337. if (status != SUCCESS) {
  338. GELOGE(FAILED, "[Parse][EngineMessage]Failed, scheduler_id_temp %s", scheduler_id_temp.c_str());
  339. REPORT_CALL_ERROR("E19999", "Parse engine message failed, scheduler_id_temp %s",
  340. scheduler_id_temp.c_str());
  341. return FAILED;
  342. }
  343. scheduler_conf.name = scheduler_utils_json[i][kName];
  344. scheduler_conf.ex_attrs = scheduler_utils_json[i][kExAttrs];
  345. scheduler_conf.cal_engines = engine_conf_map;
  346. auto it = schedulers_.find(scheduler_id_temp);
  347. if (it != schedulers_.end()) {
  348. GELOGE(FAILED, "[Check][Param]There are the same scheduler ts %s in the json file",
  349. scheduler_id_temp.c_str());
  350. REPORT_INNER_ERROR("E19999", "[Check][Param]There are the same scheduler ts %s "
  351. "in the json file", scheduler_id_temp.c_str());
  352. return FAILED;
  353. }
  354. schedulers_.emplace(scheduler_id_temp, scheduler_conf);
  355. }
  356. } catch (const nlohmann::detail::type_error &e) {
  357. GELOGE(FAILED, "[Parse][JsonFile]Failed, file %s, reason %s", path.c_str(), e.what());
  358. REPORT_CALL_ERROR("E19999", "Parse json file %s failed, reason %s", path.c_str(), e.what());
  359. return FAILED;
  360. }
  361. GELOGI("Parser json file SUCCESS");
  362. return SUCCESS;
  363. }
  364. Status DNNEngineManager::ParserEngineMessage(const json engines_json, const std::string &scheduler_mark,
  365. std::map<std::string, EngineConfPtr> &engines) {
  366. GELOGI("Begin to parser engine massage");
  367. if (engines_json.is_null()) {
  368. GELOGE(FAILED, "[Check][Param]The message of cal_engines is null");
  369. REPORT_INNER_ERROR("E19999", "The message of cal_engines is null");
  370. return FAILED;
  371. }
  372. try {
  373. if (engines_json.is_array()) {
  374. for (size_t i = 0; i < engines_json.size(); i++) {
  375. nlohmann::json engines_elems = engines_json[i];
  376. EngineConfPtr engine_conf_ptr = MakeShared<EngineConf>();
  377. if (engine_conf_ptr == nullptr) {
  378. return FAILED;
  379. }
  380. std::string engine_id = engines_elems[kId];
  381. if (!engine_id.empty()) {
  382. engine_conf_ptr->id = engine_id;
  383. } else {
  384. GELOGE(FAILED, "[Check][Param]Engine ID is null");
  385. REPORT_INNER_ERROR("E19999", "Engine ID is null");
  386. return FAILED;
  387. }
  388. if (engines_elems.find(kName) != engines_elems.end()) {
  389. engine_conf_ptr->name = engines_elems[kName];
  390. } else {
  391. GELOGW("The engine %s name is null", engine_id.c_str());
  392. }
  393. if (engines_elems.find(kIndependent) != engines_elems.end()) {
  394. engine_conf_ptr->independent = engines_elems[kIndependent];
  395. }
  396. if (engines_elems.find(kAttch) != engines_elems.end()) {
  397. engine_conf_ptr->attach = engines_elems[kAttch];
  398. }
  399. if (engines_elems.find(kSkipAssignStream) != engines_elems.end()) {
  400. engine_conf_ptr->skip_assign_stream = engines_elems[kSkipAssignStream];
  401. }
  402. engine_conf_ptr->scheduler_id = scheduler_mark;
  403. auto it = engines.find(engine_id);
  404. if (it != engines.end()) {
  405. GELOGE(FAILED, "[Check][Param]There are the same engine %s message in the json file",
  406. engine_id.c_str());
  407. REPORT_INNER_ERROR("E19999", "There are the same engine %s message in the json file",
  408. engine_id.c_str());
  409. return FAILED;
  410. }
  411. engines.emplace(engine_id, engine_conf_ptr);
  412. }
  413. } else {
  414. GELOGE(FAILED, "[Check][Param]The message of cal_engines is not array in the json file");
  415. REPORT_INNER_ERROR("E19999", "The message of cal_engines is not array in the json file");
  416. return FAILED;
  417. }
  418. } catch (const json::exception &e) {
  419. GELOGE(FAILED, "[Construct][JsonContent]Failed, reason %s", e.what());
  420. REPORT_INNER_ERROR("E19999", "Construct json content failed, reason %s", e.what());
  421. return FAILED;
  422. }
  423. GELOGI("Parser engine massage success");
  424. return SUCCESS;
  425. }
  426. Status DNNEngineManager::ReadJsonFile(const std::string &file_path, JsonHandle handle) {
  427. GELOGD("Begin to read json file");
  428. if (file_path.empty()) {
  429. GELOGE(FAILED, "[Check][Param]Json path is empty");
  430. REPORT_INNER_ERROR("E19999", "Json path is empty");
  431. return FAILED;
  432. }
  433. nlohmann::json *json_file = reinterpret_cast<nlohmann::json *>(handle);
  434. if (json_file == nullptr) {
  435. GELOGE(FAILED, "[Check][Param]Json file is nullptr");
  436. REPORT_CALL_ERROR("E19999", "Json file is nullptr");
  437. return FAILED;
  438. }
  439. const char *file = file_path.data();
  440. if ((mmAccess2(file, M_F_OK)) != EN_OK) {
  441. if (engines_map_.size() != 0) {
  442. GELOGE(FAILED, "[Check][Param]The json file %s not exists, err %s",
  443. file_path.c_str(), strerror(errno));
  444. REPORT_CALL_ERROR("E19999", "Json file %s not exists, err %s",
  445. file_path.c_str(), strerror(errno));
  446. return FAILED;
  447. } else {
  448. GELOGW("The json file %s is not needed.", file_path.c_str());
  449. return SUCCESS;
  450. }
  451. }
  452. std::ifstream ifs(file_path);
  453. if (!ifs.is_open()) {
  454. GELOGE(FAILED, "[Open][JsonFile]Failed, file %s", file_path.c_str());
  455. REPORT_CALL_ERROR("E19999", "Open json file %s failed", file_path.c_str());
  456. return FAILED;
  457. }
  458. try {
  459. ifs >> *json_file;
  460. } catch (const json::exception &e) {
  461. GELOGE(FAILED, "[Read][JsonFile]Failed, reason %s", e.what());
  462. REPORT_CALL_ERROR("E19999", "Read json file failed, reason %s", e.what());
  463. ifs.close();
  464. return FAILED;
  465. }
  466. ifs.close();
  467. GELOGD("Read json file success");
  468. return SUCCESS;
  469. }
  470. Status DNNEngineManager::CheckJsonFile() {
  471. GELOGD("Begin to check json file");
  472. for (auto &it : engines_map_) {
  473. std::string engine_name = it.first;
  474. int count = 0;
  475. for (auto &iter : schedulers_) {
  476. auto engine_map = iter.second.cal_engines;
  477. auto iter_engine_name = engine_map.find(engine_name);
  478. if (iter_engine_name != engine_map.end()) {
  479. count++;
  480. }
  481. }
  482. if (count == 0) {
  483. GELOGE(FAILED, "[Check][JsonFile]The engine message %s is not found in the json file",
  484. engine_name.c_str());
  485. REPORT_INNER_ERROR("E19999", "The engine message %s is not found in the json file",
  486. engine_name.c_str());
  487. return FAILED;
  488. }
  489. if (count > 1) {
  490. GELOGE(FAILED, "[Check][JsonFile]The same engine message %s exists in the json file",
  491. engine_name.c_str());
  492. REPORT_INNER_ERROR("E19999", "The same engine message %s exists in the json file",
  493. engine_name.c_str());
  494. return FAILED;
  495. }
  496. }
  497. GELOGD("Check json file success");
  498. return SUCCESS;
  499. }
  500. } // namespace ge

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