From: @wenfangpei Reviewed-by: Signed-off-by:tags/v1.2.0-rc1
| @@ -14,11 +14,43 @@ | |||
| # ============================================================================ | |||
| """akg process""" | |||
| import os | |||
| import shutil | |||
| import subprocess | |||
| import sys | |||
| from multiprocessing import Pool, cpu_count | |||
| def _compile_akg_task(*json_strs): | |||
| def copy_json(pid_path, ppid_path): | |||
| """ | |||
| copy json from pid_path to ppid_path | |||
| """ | |||
| if not os.path.exists(ppid_path): | |||
| os.mkdir(ppid_path) | |||
| json_files = os.listdir(pid_path) | |||
| for json_file in json_files: | |||
| shutil.move(pid_path + '/' + json_file, ppid_path) | |||
| def _compile_akg_task_gpu(*json_strs): | |||
| """ | |||
| compile func called in single process | |||
| Parameters: | |||
| json_strs: list. List contains multiple kernel infos, suitable for json compile api. | |||
| """ | |||
| p = __import__("akg", globals(), locals(), ['ms'], 0) | |||
| func = getattr(p.ms, "compilewithjson") | |||
| for json_str in json_strs: | |||
| res = func(json_str) | |||
| if not res: | |||
| raise ValueError("Compile error, args: {}!".format(json_str)) | |||
| pid_path = os.path.realpath("./cuda_meta_" + str(os.getpid())) | |||
| if os.path.exists(pid_path): | |||
| copy_json(pid_path, os.path.realpath("./cuda_meta_" + str(os.getppid()))) | |||
| shutil.rmtree(pid_path) | |||
| def _compile_akg_task_ascend(*json_strs): | |||
| """ | |||
| compile func called in single process | |||
| @@ -28,24 +60,26 @@ def _compile_akg_task(*json_strs): | |||
| akg_compiler = os.path.join(os.path.split( | |||
| os.path.realpath(__file__))[0], "compiler.py") | |||
| for json_str in json_strs: | |||
| res = subprocess.run( | |||
| [sys.executable, akg_compiler, json_str], text=True) | |||
| res = subprocess.run([sys.executable, akg_compiler, json_str], text=True) | |||
| if res.returncode != 0: | |||
| raise ValueError("Failed, args: {}!".format(json_str)) | |||
| def create_akg_parallel_process(process_num, wait_time): | |||
| def create_akg_parallel_process(process_num, wait_time, platform=""): | |||
| """ | |||
| create AkgParallelCompiler object | |||
| Returns: | |||
| AkgParallelCompiler | |||
| """ | |||
| return AkgProcess(process_num, wait_time) | |||
| return AkgProcess(process_num, wait_time, platform) | |||
| class AkgProcess: | |||
| """akg kernel parallel process""" | |||
| def __init__(self, process_num, wait_time): | |||
| def __init__(self, process_num, wait_time, platform=""): | |||
| """ | |||
| Args: | |||
| process_num: int. processes number | |||
| @@ -61,6 +95,7 @@ class AkgProcess: | |||
| self.process_num = min([cpu_count(), max_proc_num, process_num]) | |||
| self.args = [[] for _ in range(self.process_num)] | |||
| self.wait_time = wait_time | |||
| self.platform = platform | |||
| self.argc = 0 | |||
| def compile(self): | |||
| @@ -71,9 +106,16 @@ class AkgProcess: | |||
| """ | |||
| if self.argc == 0: | |||
| raise ValueError("json must be not null") | |||
| with Pool(processes=self.process_num) as pool: | |||
| res = pool.starmap_async(_compile_akg_task, self.args) | |||
| res.get(timeout=self.wait_time) | |||
| if self.platform == "GPU": | |||
| with Pool(processes=self.process_num) as pool: | |||
| res = pool.starmap_async(_compile_akg_task_gpu, self.args) | |||
| res.get(timeout=self.wait_time) | |||
| elif self.platform == "ASCEND": | |||
| with Pool(processes=self.process_num) as pool: | |||
| res = pool.starmap_async(_compile_akg_task_ascend, self.args) | |||
| res.get(timeout=self.wait_time) | |||
| else: | |||
| raise ValueError("The value of 'platform' must be 'GPU' or 'ASCEND'.") | |||
| return True | |||
| def accept_json(self, json): | |||
| @@ -1,4 +1,4 @@ | |||
| # Copyright 2020 Huawei Technologies Co., Ltd | |||
| # Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| # | |||
| # Licensed under the Apache License, Version 2.0 (the "License"); | |||
| # you may not use this file except in compliance with the License. | |||
| @@ -15,6 +15,22 @@ | |||
| """kernel build server""" | |||
| import os | |||
| from mindspore import log as logger | |||
| from mindspore._extends.parallel_compile.akg_compiler.akg_process import create_akg_parallel_process | |||
| class AkgBuilder: | |||
| """Akg building wrapper""" | |||
| def __init__(self): | |||
| pass | |||
| def create(self, process_num, waitime, platform=""): | |||
| self.akg_builder = create_akg_parallel_process(process_num, waitime, platform) | |||
| def accept_json(self, json): | |||
| return self.akg_builder.accept_json(json) | |||
| def compile(self): | |||
| return self.akg_builder.compile() | |||
| class Messager: | |||
| '''Messager''' | |||
| @@ -1,4 +1,4 @@ | |||
| # Copyright 2020 Huawei Technologies Co., Ltd | |||
| # Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| # | |||
| # Licensed under the Apache License, Version 2.0 (the "License"); | |||
| # you may not use this file except in compliance with the License. | |||
| @@ -14,9 +14,9 @@ | |||
| # ============================================================================ | |||
| """kernel build server for ascend""" | |||
| import sys | |||
| from mindspore._extends.remote.kernel_build_server import Messager, get_logger | |||
| from mindspore._extends.parallel_compile.tbe_compiler.tbe_process import create_tbe_parallel_process, op_select_format, check_supported | |||
| from mindspore._extends.parallel_compile.akg_compiler.akg_process import create_akg_parallel_process | |||
| from mindspore._extends.remote.kernel_build_server import Messager, get_logger, AkgBuilder | |||
| from mindspore._extends.parallel_compile.tbe_compiler.tbe_process import create_tbe_parallel_process, op_select_format | |||
| from mindspore._extends.parallel_compile.tbe_compiler.tbe_process import check_supported | |||
| class TbeBuilder: | |||
| """Tbe building wrapper""" | |||
| @@ -42,21 +42,6 @@ class TbeBuilder: | |||
| def exit(self): | |||
| self.tbe_builder.exit() | |||
| class AkgBuilder: | |||
| """Akg building wrapper""" | |||
| def __init__(self): | |||
| pass | |||
| def create(self, process_num, waitime): | |||
| self.akg_builder = create_akg_parallel_process(process_num, waitime) | |||
| def accept_json(self, json): | |||
| return self.akg_builder.accept_json(json) | |||
| def compile(self): | |||
| return self.akg_builder.compile() | |||
| class AscendMessager(Messager): | |||
| ''' | |||
| Ascend Messager | |||
| @@ -112,7 +97,7 @@ class AscendMessager(Messager): | |||
| process_num_str = self.get_message() | |||
| self.send_ack() | |||
| wait_time_str = self.get_message() | |||
| self.akg_builder.create(int(process_num_str), int(wait_time_str)) | |||
| self.akg_builder.create(int(process_num_str), int(wait_time_str), "ASCEND") | |||
| self.send_ack() | |||
| elif arg == 'AKG/DATA': | |||
| self.send_ack() | |||
| @@ -1,4 +1,4 @@ | |||
| # Copyright 2020 Huawei Technologies Co., Ltd | |||
| # Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| # | |||
| # Licensed under the Apache License, Version 2.0 (the "License"); | |||
| # you may not use this file except in compliance with the License. | |||
| @@ -15,7 +15,7 @@ | |||
| """kernel build server for gpu""" | |||
| import os | |||
| import sys | |||
| from mindspore._extends.remote.kernel_build_server import Messager, get_logger | |||
| from mindspore._extends.remote.kernel_build_server import Messager, get_logger, AkgBuilder | |||
| from mindspore._extends.parallel_compile.akg_compiler.compiler import run_compiler as akg_compile_single | |||
| class GpuMessager(Messager): | |||
| @@ -27,6 +27,7 @@ class GpuMessager(Messager): | |||
| def __init__(self, fdin, fdout): | |||
| super().__init__(fdin, fdout) | |||
| get_logger().info("[TRACE] GPU Messager init...") | |||
| self.akg_builder = AkgBuilder() | |||
| def handle(self): | |||
| """ | |||
| @@ -36,6 +37,27 @@ class GpuMessager(Messager): | |||
| arg = self.get_message() | |||
| if arg == 'AKG/PID': | |||
| self.send_res(os.getpid()) | |||
| elif arg == 'AKG/START': | |||
| self.send_ack() | |||
| process_num_str = self.get_message() | |||
| self.send_ack() | |||
| wait_time_str = self.get_message() | |||
| self.akg_builder.create(int(process_num_str), int(wait_time_str), "GPU") | |||
| self.send_ack() | |||
| elif arg == 'AKG/DATA': | |||
| self.send_ack() | |||
| while True: | |||
| req = self.get_message() | |||
| if req.startswith('{'): | |||
| self.akg_builder.accept_json(req) | |||
| self.send_ack() | |||
| elif req == 'AKG/WAIT': | |||
| res = self.akg_builder.compile() | |||
| self.send_res(res) | |||
| break | |||
| else: | |||
| self.send_ack(False) | |||
| break | |||
| elif arg == 'AKG/COMPILE': | |||
| self.send_ack() | |||
| json = self.get_message() | |||
| @@ -113,6 +113,7 @@ if(ENABLE_GPU) | |||
| "backend/kernel_compiler/akg/akg_kernel_json_generator.cc" | |||
| "backend/kernel_compiler/akg/akg_kernel_json_decoder.cc" | |||
| "backend/kernel_compiler/akg/akg_kernel_attrs_process.cc" | |||
| "backend/kernel_compiler/akg/akg_kernel_build.cc" | |||
| ) | |||
| list(APPEND CUDA_NVCC_FLAGS -arch=sm_53 --expt-relaxed-constexpr) | |||
| @@ -13,6 +13,7 @@ if(ENABLE_D) | |||
| file(GLOB_RECURSE D_SRC_LIST RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} | |||
| "kernel_query.cc" | |||
| "kernel_fusion.cc" | |||
| "akg/akg_kernel_build.cc" | |||
| "akg/ascend/*.cc" | |||
| "akg/akg_kernel_json_generator.cc" | |||
| "akg/akg_kernel_json_decoder.cc" | |||
| @@ -74,6 +75,7 @@ endif() | |||
| if(ENABLE_GPU) | |||
| file(GLOB_RECURSE CUDA_SRC_LIST RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} | |||
| "gpu/*.cu" | |||
| "akg/akg_kernel_build.cc" | |||
| "akg/gpu/*.cc" | |||
| "akg/akg_kernel_json_generator.cc" | |||
| "akg/akg_kernel_json_decoder.cc" | |||
| @@ -0,0 +1,172 @@ | |||
| /** | |||
| * Copyright 2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| * You may obtain a copy of the License at | |||
| * | |||
| * http://www.apache.org/licenses/LICENSE-2.0 | |||
| * | |||
| * Unless required by applicable law or agreed to in writing, software | |||
| * distributed under the License is distributed on an "AS IS" BASIS, | |||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
| * See the License for the specific language governing permissions and | |||
| * limitations under the License. | |||
| */ | |||
| #include "backend/kernel_compiler/akg/akg_kernel_build.h" | |||
| #include <algorithm> | |||
| #include <map> | |||
| #include <memory> | |||
| #include <string> | |||
| #include <unordered_set> | |||
| #include <utility> | |||
| #include <vector> | |||
| #include "ir/dtype.h" | |||
| #include "ir/func_graph.h" | |||
| #include "backend/kernel_compiler/common_utils.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_json_generator.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_attrs_process.h" | |||
| #include "backend/session/anf_runtime_algorithm.h" | |||
| namespace mindspore { | |||
| namespace kernel { | |||
| constexpr int32_t PROCESS_NUM = 16; | |||
| constexpr int32_t TIME_OUT = 300; | |||
| std::vector<std::string> AkgKernelBuilder::GetNotCachedKernelJsons(const std::vector<JsonNodePair> &build_args) { | |||
| // Remove cached nodes, gether unique nodes, and collect repeated nodes which need postprecess. | |||
| std::vector<std::string> jsons; | |||
| std::unordered_set<std::string> kernel_name_set; | |||
| for (const auto &[json_generator, anf_node] : build_args) { | |||
| MS_EXCEPTION_IF_NULL(anf_node); | |||
| auto kernel_name = json_generator.kernel_name(); | |||
| MS_LOG(DEBUG) << "Akg start compile op: " << kernel_name; | |||
| auto cached_kernel_pack = AkgSearchCache(kernel_name, GetProcessorStr(anf_node)); | |||
| if (cached_kernel_pack != nullptr) { | |||
| MS_LOG(DEBUG) << "Use cached kernel, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| AkgSetKernelMod(cached_kernel_pack, json_generator, anf_node); | |||
| continue; | |||
| } | |||
| if (kernel_name_set.count(kernel_name) != 0) { | |||
| repeat_nodes_.push_back({json_generator, anf_node}); | |||
| continue; | |||
| } | |||
| kernel_name_set.insert(kernel_name); | |||
| auto kernel_json = json_generator.kernel_json_str(); | |||
| AkgSaveJsonInfo(kernel_name, kernel_json); | |||
| jsons.push_back(kernel_json); | |||
| } | |||
| return jsons; | |||
| } | |||
| bool AkgKernelBuilder::InsertToCache(const std::vector<JsonNodePair> &build_args) { | |||
| for (const auto &[json_generator, anf_node] : build_args) { | |||
| auto kernel_name = json_generator.kernel_name(); | |||
| auto new_kernel_pack = AkgInsertCache(kernel_name, GetProcessorStr(anf_node)); | |||
| if (new_kernel_pack == nullptr) { | |||
| MS_LOG(ERROR) << "Insert to cache failed, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| return false; | |||
| } | |||
| AkgSetKernelMod(new_kernel_pack, json_generator, anf_node); | |||
| MS_LOG(DEBUG) << "Akg compile " << kernel_name << " kernel and insert cache successfully!"; | |||
| } | |||
| return true; | |||
| } | |||
| bool AkgKernelBuilder::HandleRepeatNodes() { | |||
| for (const auto &[json_generator, anf_node] : repeat_nodes_) { | |||
| auto kernel_name = json_generator.kernel_name(); | |||
| auto cached_kernel_pack = AkgSearchCache(kernel_name, GetProcessorStr(anf_node)); | |||
| if (cached_kernel_pack == nullptr) { | |||
| MS_LOG(ERROR) << "Use cached kernel failed, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| return false; | |||
| } | |||
| MS_LOG(INFO) << "Use just compiled kernel, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| AkgSetKernelMod(cached_kernel_pack, json_generator, anf_node); | |||
| } | |||
| return true; | |||
| } | |||
| bool AkgKernelBuilder::AkgOpParallelBuild(const std::vector<JsonNodePair> &build_args) { | |||
| repeat_nodes_.clear(); | |||
| auto jsons = GetNotCachedKernelJsons(build_args); | |||
| if (jsons.empty()) { | |||
| return true; | |||
| } | |||
| kernel::KernelBuildClient *client = GetClient(); | |||
| if (!client->AkgStart(PROCESS_NUM, TIME_OUT)) { | |||
| MS_LOG(ERROR) << "Akg start failed."; | |||
| return false; | |||
| } | |||
| if (!client->AkgSendData(jsons)) { | |||
| MS_LOG(ERROR) << "Akg send data failed."; | |||
| return false; | |||
| } | |||
| if (!client->AkgWait()) { | |||
| MS_LOG(ERROR) << "Akg compile failed."; | |||
| return false; | |||
| } | |||
| // All unique done here, cache them and set kernel. | |||
| if (!InsertToCache(build_args)) { | |||
| MS_LOG(ERROR) << "Insert cache failed."; | |||
| return false; | |||
| } | |||
| if (!HandleRepeatNodes()) { | |||
| MS_LOG(ERROR) << "Handle repeat nodes failed."; | |||
| return false; | |||
| } | |||
| return true; | |||
| } | |||
| bool AkgKernelBuilder::AkgKernelParallelBuild(const std::vector<AnfNodePtr> &anf_nodes) { | |||
| std::vector<JsonNodePair> json_and_node; | |||
| for (const auto &anf_node : anf_nodes) { | |||
| MS_EXCEPTION_IF_NULL(anf_node); | |||
| AkgKernelJsonGenerator akg_kernel_json_generator; | |||
| auto cnode = anf_node->cast<CNodePtr>(); | |||
| MS_EXCEPTION_IF_NULL(cnode); | |||
| if (AnfAlgo::IsGraphKernel(cnode)) { | |||
| auto func_graph = AnfAlgo::GetCNodeFuncGraphPtr(cnode); | |||
| MS_EXCEPTION_IF_NULL(func_graph); | |||
| auto mng = func_graph->manager(); | |||
| if (mng == nullptr) { | |||
| mng = Manage(func_graph, true); | |||
| func_graph->set_manager(mng); | |||
| } | |||
| std::vector<AnfNodePtr> node_list, input_list, output_list; | |||
| MS_LOG(INFO) << "Akg start compile composite op[" << anf_node->fullname_with_scope() << "]"; | |||
| GetValidKernelNodes(func_graph, &node_list, &input_list, &output_list); | |||
| if (!akg_kernel_json_generator.CollectFusedJson(node_list, input_list, output_list)) { | |||
| MS_EXCEPTION(UnknownError) << "Akg build failed composite op[" << anf_node->fullname_with_scope() << "]."; | |||
| } | |||
| } else { | |||
| if (!akg_kernel_json_generator.CollectJson(anf_node)) { | |||
| MS_EXCEPTION(UnknownError) << "Akg build failed basic op[" << anf_node->fullname_with_scope() << "]."; | |||
| } | |||
| } | |||
| json_and_node.push_back({akg_kernel_json_generator, anf_node}); | |||
| } | |||
| if (json_and_node.empty()) { | |||
| MS_LOG(DEBUG) << "There is no kernel needed to be compiled."; | |||
| return true; | |||
| } | |||
| bool res = AkgOpParallelBuild(json_and_node); | |||
| if (!res) { | |||
| MS_LOG(ERROR) << "Akg-Op Parallel Building fail."; | |||
| } | |||
| return true; | |||
| } | |||
| } // namespace kernel | |||
| } // namespace mindspore | |||
| @@ -0,0 +1,57 @@ | |||
| /** | |||
| * Copyright 2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| * You may obtain a copy of the License at | |||
| * | |||
| * http://www.apache.org/licenses/LICENSE-2.0 | |||
| * | |||
| * Unless required by applicable law or agreed to in writing, software | |||
| * distributed under the License is distributed on an "AS IS" BASIS, | |||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
| * See the License for the specific language governing permissions and | |||
| * limitations under the License. | |||
| */ | |||
| #ifndef MINDSPORE_CCSRC_BACKEND_KERNEL_COMPILER_AKG_AKG_KERNEL_BUILD_H_ | |||
| #define MINDSPORE_CCSRC_BACKEND_KERNEL_COMPILER_AKG_AKG_KERNEL_BUILD_H_ | |||
| #include <string> | |||
| #include <utility> | |||
| #include <vector> | |||
| #include <map> | |||
| #include "ir/anf.h" | |||
| #include "backend/kernel_compiler/kernel.h" | |||
| #include "backend/session/kernel_build_client.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_json_generator.h" | |||
| namespace mindspore { | |||
| namespace kernel { | |||
| using JsonNodePair = std::pair<AkgKernelJsonGenerator, AnfNodePtr>; | |||
| class AkgKernelBuilder { | |||
| public: | |||
| AkgKernelBuilder() = default; | |||
| ~AkgKernelBuilder() = default; | |||
| virtual KernelBuildClient *GetClient() = 0; | |||
| virtual KernelPackPtr AkgSearchCache(const std::string &kernel_name, const std::string &processor) = 0; | |||
| virtual KernelPackPtr AkgInsertCache(const std::string &kernel_name, const std::string &processor) = 0; | |||
| virtual void AkgSetKernelMod(const KernelPackPtr &kernel_pack, const AkgKernelJsonGenerator &json_generator, | |||
| const AnfNodePtr &anf_node) = 0; | |||
| virtual void AkgSaveJsonInfo(const string &kernel_name, const string &kernel_json) = 0; | |||
| bool AkgKernelParallelBuild(const std::vector<AnfNodePtr> &anf_nodes); | |||
| private: | |||
| std::vector<std::string> GetNotCachedKernelJsons(const std::vector<JsonNodePair> &build_args); | |||
| bool InsertToCache(const std::vector<JsonNodePair> &build_args); | |||
| bool HandleRepeatNodes(); | |||
| bool AkgOpParallelBuild(const std::vector<JsonNodePair> &build_args); | |||
| std::vector<JsonNodePair> repeat_nodes_; | |||
| }; | |||
| } // namespace kernel | |||
| } // namespace mindspore | |||
| #endif // MINDSPORE_CCSRC_BACKEND_KERNEL_COMPILER_AKG_AKG_KERNEL_BUILD_H_ | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2020 Huawei Technologies Co., Ltd | |||
| * Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -25,160 +25,32 @@ | |||
| #include <vector> | |||
| #include "ir/dtype.h" | |||
| #include "ir/func_graph.h" | |||
| #include "backend/kernel_compiler/kernel.h" | |||
| #include "backend/kernel_compiler/common_utils.h" | |||
| #include "backend/kernel_compiler/tbe/tbe_utils.h" | |||
| #include "backend/kernel_compiler/akg/ascend/akg_ascend_kernel_mod.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_json_generator.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_attrs_process.h" | |||
| #include "backend/session/anf_runtime_algorithm.h" | |||
| #include "backend/session/kernel_build_client.h" | |||
| namespace mindspore { | |||
| namespace kernel { | |||
| namespace { | |||
| constexpr int32_t PROCESS_NUM = 16; | |||
| constexpr int32_t TIME_OUT = 300; | |||
| KernelPackPtr AkgAscendKernelBuilder::AkgSearchCache(const std::string &kernel_name, const std::string &processor) { | |||
| return tbe::TbeUtils::SearchCache(kernel_name, processor); | |||
| } | |||
| KernelPackPtr AkgAscendKernelBuilder::AkgInsertCache(const std::string &kernel_name, const std::string &processor) { | |||
| return tbe::TbeUtils::InsertCache(kernel_name, processor); | |||
| } | |||
| void SetKernelMod(const KernelPackPtr &kernel_pack, const AkgKernelJsonGenerator &json_generator, | |||
| const AnfNodePtr &anf_node) { | |||
| void AkgAscendKernelBuilder::AkgSetKernelMod(const KernelPackPtr &kernel_pack, | |||
| const AkgKernelJsonGenerator &json_generator, const AnfNodePtr &anf_node) { | |||
| auto kernel_mod_ptr = std::make_shared<AkgKernelMod>(kernel_pack); | |||
| kernel_mod_ptr->SetInputSizeList(json_generator.input_size_list()); | |||
| kernel_mod_ptr->SetOutputSizeList(json_generator.output_size_list()); | |||
| AnfAlgo::SetKernelMod(kernel_mod_ptr, anf_node.get()); | |||
| } | |||
| } // namespace | |||
| std::vector<std::string> AkgAscendKernelBuilder::GetNotCachedKernelJsons(const std::vector<JsonNodePair> &build_args) { | |||
| // Remove cached nodes, gether unique nodes, and collect repeated nodes which need postprecess. | |||
| std::vector<std::string> jsons; | |||
| std::unordered_set<std::string> kernel_name_set; | |||
| for (const auto &[json_generator, anf_node] : build_args) { | |||
| MS_EXCEPTION_IF_NULL(anf_node); | |||
| auto kernel_name = json_generator.kernel_name(); | |||
| MS_LOG(DEBUG) << "Akg start compile op: " << kernel_name; | |||
| auto cached_kernel_pack = tbe::TbeUtils::SearchCache(kernel_name, GetProcessorStr(anf_node)); | |||
| if (cached_kernel_pack != nullptr) { | |||
| MS_LOG(DEBUG) << "Use cached kernel, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| SetKernelMod(cached_kernel_pack, json_generator, anf_node); | |||
| continue; | |||
| } | |||
| if (kernel_name_set.count(kernel_name) != 0) { | |||
| repeat_nodes_.push_back({json_generator, anf_node}); | |||
| continue; | |||
| } | |||
| kernel_name_set.insert(kernel_name); | |||
| auto kernel_json = json_generator.kernel_json_str(); | |||
| kernel::SaveJsonInfo(kernel_name, kernel_json); | |||
| jsons.push_back(kernel_json); | |||
| } | |||
| return jsons; | |||
| } | |||
| bool AkgAscendKernelBuilder::InsertToCache(const std::vector<JsonNodePair> &build_args) { | |||
| for (const auto &[json_generator, anf_node] : build_args) { | |||
| auto kernel_name = json_generator.kernel_name(); | |||
| auto new_kernel_pack = tbe::TbeUtils::InsertCache(kernel_name, GetProcessorStr(anf_node)); | |||
| if (new_kernel_pack == nullptr) { | |||
| MS_LOG(ERROR) << "Insert to cache failed, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| return false; | |||
| } | |||
| SetKernelMod(new_kernel_pack, json_generator, anf_node); | |||
| MS_LOG(DEBUG) << "Akg compile " << kernel_name << " kernel and insert cache successfully!"; | |||
| } | |||
| return true; | |||
| } | |||
| bool AkgAscendKernelBuilder::HandleRepeatNodes() { | |||
| for (const auto &[json_generator, anf_node] : repeat_nodes_) { | |||
| auto kernel_name = json_generator.kernel_name(); | |||
| auto cached_kernel_pack = tbe::TbeUtils::SearchCache(kernel_name, GetProcessorStr(anf_node)); | |||
| if (cached_kernel_pack == nullptr) { | |||
| MS_LOG(ERROR) << "Use cached kernel failed, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| return false; | |||
| } | |||
| MS_LOG(INFO) << "Use just compiled kernel, kernel_name[" << kernel_name << "], fullname_with_scope[" | |||
| << anf_node->fullname_with_scope() << "]."; | |||
| SetKernelMod(cached_kernel_pack, json_generator, anf_node); | |||
| } | |||
| return true; | |||
| } | |||
| bool AkgAscendKernelBuilder::AkgOpParallelBuild(const std::vector<JsonNodePair> &build_args) { | |||
| repeat_nodes_.clear(); | |||
| auto jsons = GetNotCachedKernelJsons(build_args); | |||
| if (jsons.empty()) { | |||
| return true; | |||
| } | |||
| // Start building in AKG | |||
| if (!AscendKernelBuildClient::Instance().AkgStart(PROCESS_NUM, TIME_OUT)) { | |||
| MS_LOG(ERROR) << "Akg start failed."; | |||
| return false; | |||
| } | |||
| if (!AscendKernelBuildClient::Instance().AkgSendData(jsons)) { | |||
| MS_LOG(ERROR) << "Akg send data failed."; | |||
| return false; | |||
| } | |||
| if (!AscendKernelBuildClient::Instance().AkgWait()) { | |||
| MS_LOG(ERROR) << "Akg compile failed."; | |||
| return false; | |||
| } | |||
| // All unique done here, cache them and set kernel. | |||
| if (!InsertToCache(build_args)) { | |||
| MS_LOG(ERROR) << "Insert cache failed."; | |||
| return false; | |||
| } | |||
| if (!HandleRepeatNodes()) { | |||
| MS_LOG(ERROR) << "Handle repeat nodes failed."; | |||
| return false; | |||
| } | |||
| return true; | |||
| } | |||
| bool AkgAscendKernelParallelBuild(const std::vector<AnfNodePtr> &anf_nodes) { | |||
| std::vector<JsonNodePair> json_and_node; | |||
| for (const auto &anf_node : anf_nodes) { | |||
| MS_EXCEPTION_IF_NULL(anf_node); | |||
| AkgKernelJsonGenerator akg_kernel_json_generator; | |||
| auto cnode = anf_node->cast<CNodePtr>(); | |||
| MS_EXCEPTION_IF_NULL(cnode); | |||
| if (AnfAlgo::IsGraphKernel(cnode)) { | |||
| auto func_graph = AnfAlgo::GetCNodeFuncGraphPtr(cnode); | |||
| MS_EXCEPTION_IF_NULL(func_graph); | |||
| auto mng = func_graph->manager(); | |||
| if (mng == nullptr) { | |||
| mng = Manage(func_graph, true); | |||
| func_graph->set_manager(mng); | |||
| } | |||
| std::vector<AnfNodePtr> node_list, input_list, output_list; | |||
| MS_LOG(INFO) << "Akg start compile composite op[" << anf_node->fullname_with_scope() << "]"; | |||
| GetValidKernelNodes(func_graph, &node_list, &input_list, &output_list); | |||
| if (!akg_kernel_json_generator.CollectFusedJson(node_list, input_list, output_list)) { | |||
| MS_EXCEPTION(UnknownError) << "Akg build failed composite op[" << anf_node->fullname_with_scope() << "]."; | |||
| } | |||
| } else { | |||
| if (!akg_kernel_json_generator.CollectJson(anf_node)) { | |||
| MS_EXCEPTION(UnknownError) << "Akg build failed basic op[" << anf_node->fullname_with_scope() << "]."; | |||
| } | |||
| } | |||
| json_and_node.push_back({akg_kernel_json_generator, anf_node}); | |||
| } | |||
| if (json_and_node.empty()) { | |||
| MS_LOG(DEBUG) << "There is no kernel needed to be compiled."; | |||
| return true; | |||
| } | |||
| AkgAscendKernelBuilder akg_ascend_kernel_builder; | |||
| return akg_ascend_kernel_builder.AkgOpParallelBuild(json_and_node); | |||
| void AkgAscendKernelBuilder::AkgSaveJsonInfo(const string &kernel_name, const string &kernel_json) { | |||
| kernel::SaveJsonInfo(kernel_name, kernel_json); | |||
| } | |||
| } // namespace kernel | |||
| } // namespace mindspore | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2020 Huawei Technologies Co., Ltd | |||
| * Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -22,28 +22,22 @@ | |||
| #include <vector> | |||
| #include <map> | |||
| #include "ir/anf.h" | |||
| #include "backend/kernel_compiler/kernel.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_json_generator.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_build.h" | |||
| namespace mindspore { | |||
| namespace kernel { | |||
| using JsonNodePair = std::pair<AkgKernelJsonGenerator, AnfNodePtr>; | |||
| class AkgAscendKernelBuilder { | |||
| class AkgAscendKernelBuilder : public AkgKernelBuilder { | |||
| public: | |||
| AkgAscendKernelBuilder() = default; | |||
| ~AkgAscendKernelBuilder() = default; | |||
| bool AkgOpParallelBuild(const std::vector<JsonNodePair> &build_args); | |||
| private: | |||
| std::vector<std::string> GetNotCachedKernelJsons(const std::vector<JsonNodePair> &build_args); | |||
| bool InsertToCache(const std::vector<JsonNodePair> &build_args); | |||
| bool HandleRepeatNodes(); | |||
| std::vector<JsonNodePair> repeat_nodes_; | |||
| kernel::KernelBuildClient *GetClient() override { return &(kernel::AscendKernelBuildClient::Instance()); } | |||
| KernelPackPtr AkgSearchCache(const std::string &kernel_name, const std::string &processor) override; | |||
| KernelPackPtr AkgInsertCache(const std::string &kernel_name, const std::string &processor) override; | |||
| void AkgSetKernelMod(const KernelPackPtr &kernel_pack, const AkgKernelJsonGenerator &json_generator, | |||
| const AnfNodePtr &anf_node) override; | |||
| void AkgSaveJsonInfo(const string &kernel_name, const string &kernel_json) override; | |||
| }; | |||
| bool AkgAscendKernelParallelBuild(const std::vector<AnfNodePtr> &anf_nodes); | |||
| } // namespace kernel | |||
| } // namespace mindspore | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2020 Huawei Technologies Co., Ltd | |||
| * Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -19,19 +19,36 @@ | |||
| #include <vector> | |||
| #include <memory> | |||
| #include <string> | |||
| #include "backend/kernel_compiler/kernel.h" | |||
| #include "backend/kernel_compiler/common_utils.h" | |||
| #include "backend/kernel_compiler/akg/gpu/akg_gpu_kernel_mod.h" | |||
| #include "utils/ms_utils.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_json_generator.h" | |||
| #include "backend/session/anf_runtime_algorithm.h" | |||
| #include "backend/session/kernel_build_client.h" | |||
| namespace mindspore { | |||
| namespace kernel { | |||
| constexpr int32_t ARGS_SIZE = 1; | |||
| constexpr auto kCompileWithJsonFunc = "compilewithjson"; | |||
| KernelPackPtr AkgGpuKernelBuilder::AkgSearchCache(const std::string &kernel_name, const std::string &processor) { | |||
| return SearchCache(kernel_name, processor); | |||
| } | |||
| KernelPackPtr AkgGpuKernelBuilder::AkgInsertCache(const std::string &kernel_name, const std::string &processor) { | |||
| return InsertCache(kernel_name, processor); | |||
| } | |||
| void AkgGpuKernelBuilder::AkgSetKernelMod(const KernelPackPtr &kernel_pack, | |||
| const AkgKernelJsonGenerator &json_generator, const AnfNodePtr &anf_node) { | |||
| auto kernel_mod_ptr = std::make_shared<GpuKernelMod>(kernel_pack); | |||
| kernel_mod_ptr->SetInputSizeList(json_generator.input_size_list()); | |||
| kernel_mod_ptr->SetOutputSizeList(json_generator.output_size_list()); | |||
| AnfAlgo::SetKernelMod(kernel_mod_ptr, anf_node.get()); | |||
| } | |||
| void AkgGpuKernelBuilder::AkgSaveJsonInfo(const string &kernel_name, const string &kernel_json) { | |||
| kernel::SaveJsonInfo(kernel_name, kernel_json, kernel::KernelMeta::GetInstance()->kernel_meta_path()); | |||
| } | |||
| KernelPackPtr AkgGpuKernelBuilder::OpBuild(const AkgKernelJsonGenerator &json_generator, const AnfNodePtr &anf_node) { | |||
| MS_EXCEPTION_IF_NULL(anf_node); | |||
| auto processor = GetProcessorStr(anf_node); | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2020 Huawei Technologies Co., Ltd | |||
| * Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -17,17 +17,22 @@ | |||
| #ifndef MINDSPORE_CCSRC_BACKEND_KERNEL_COMPILER_AKG_GPU_AKG_GPU_KERNEL_BUILD_H_ | |||
| #define MINDSPORE_CCSRC_BACKEND_KERNEL_COMPILER_AKG_GPU_AKG_GPU_KERNEL_BUILD_H_ | |||
| #include <string> | |||
| #include "backend/kernel_compiler/kernel.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_json_generator.h" | |||
| #include "backend/kernel_compiler/akg/akg_kernel_build.h" | |||
| #include "base/base.h" | |||
| namespace mindspore { | |||
| namespace kernel { | |||
| class AkgGpuKernelBuilder { | |||
| class AkgGpuKernelBuilder : public AkgKernelBuilder { | |||
| public: | |||
| AkgGpuKernelBuilder() = default; | |||
| ~AkgGpuKernelBuilder() = default; | |||
| kernel::KernelBuildClient *GetClient() override { return &(kernel::GpuKernelBuildClient::Instance()); } | |||
| KernelPackPtr AkgSearchCache(const std::string &kernel_name, const std::string &processor) override; | |||
| KernelPackPtr AkgInsertCache(const std::string &kernel_name, const std::string &processor) override; | |||
| void AkgSetKernelMod(const KernelPackPtr &kernel_pack, const AkgKernelJsonGenerator &json_generator, | |||
| const AnfNodePtr &anf_node) override; | |||
| void AkgSaveJsonInfo(const string &kernel_name, const string &kernel_json) override; | |||
| KernelModPtr BuildByJson(const AnfNodePtr &anf_node); | |||
| KernelModPtr FuseByJson(const AnfNodePtr &anf_node); | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2020 Huawei Technologies Co., Ltd | |||
| * Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -28,6 +28,54 @@ void ReplaceStr(std::string *dest, const std::string &replace, char new_char) { | |||
| } | |||
| } | |||
| bool KernelBuildClient::AkgStart(int process_num, int wait_time) { | |||
| // Start compiling.. | |||
| auto res = SendRequest(kAkgStart); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/START failed, res: " << res; | |||
| return false; | |||
| } | |||
| std::string process_num_str = std::to_string(process_num); | |||
| res = SendRequest(process_num_str); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/START(process_num) responds failed, res: " << res; | |||
| return false; | |||
| } | |||
| std::string wait_time_str = std::to_string(wait_time); | |||
| res = SendRequest(wait_time_str); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/START(wait_time) responds failed, res: " << res; | |||
| return false; | |||
| } | |||
| return true; | |||
| } | |||
| bool KernelBuildClient::AkgSendData(const std::vector<std::string> &jsons) { | |||
| auto res = SendRequest(kAkgData); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/DATA failed, res: " << res; | |||
| return false; | |||
| } | |||
| for (auto &json : jsons) { | |||
| res = SendRequest(json); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/DATA.. responds failed, res: " << res << ", when sending [" << json << "]"; | |||
| return false; | |||
| } | |||
| } | |||
| return true; | |||
| } | |||
| // Fetch the result of AKG compiling. | |||
| bool KernelBuildClient::AkgWait() { | |||
| auto res = SendRequest(kAkgWait); | |||
| if (res != kTrue) { | |||
| MS_LOG(ERROR) << "AKG/WAIT failed, res: " << res; | |||
| return false; | |||
| } | |||
| return true; | |||
| } | |||
| bool AscendKernelBuildClient::TbePre(const std::string &mode) { | |||
| auto res = SendRequest(kTbePre); | |||
| if (res.find(kSuccess) == res.npos) { | |||
| @@ -95,54 +143,6 @@ void AscendKernelBuildClient::TbeReset() { | |||
| } | |||
| } | |||
| bool AscendKernelBuildClient::AkgStart(int process_num, int wait_time) { | |||
| // Start compiling.. | |||
| auto res = SendRequest(kAkgStart); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/START failed, res: " << res; | |||
| return false; | |||
| } | |||
| std::string process_num_str = std::to_string(process_num); | |||
| res = SendRequest(process_num_str); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/START(process_num) responds failed, res: " << res; | |||
| return false; | |||
| } | |||
| std::string wait_time_str = std::to_string(wait_time); | |||
| res = SendRequest(wait_time_str); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/START(wait_time) responds failed, res: " << res; | |||
| return false; | |||
| } | |||
| return true; | |||
| } | |||
| bool AscendKernelBuildClient::AkgSendData(const std::vector<std::string> &jsons) { | |||
| auto res = SendRequest(kAkgData); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/DATA failed, res: " << res; | |||
| return false; | |||
| } | |||
| for (auto &json : jsons) { | |||
| res = SendRequest(json); | |||
| if (res != kAck) { | |||
| MS_LOG(ERROR) << "AKG/DATA.. responds failed, res: " << res << ", when sending [" << json << "]"; | |||
| return false; | |||
| } | |||
| } | |||
| return true; | |||
| } | |||
| // Fetch the result of AKG compiling. | |||
| bool AscendKernelBuildClient::AkgWait() { | |||
| auto res = SendRequest(kAkgWait); | |||
| if (res != kTrue) { | |||
| MS_LOG(ERROR) << "AKG/WAIT failed, res: " << res; | |||
| return false; | |||
| } | |||
| return true; | |||
| } | |||
| std::string AscendKernelBuildClient::SelectFormat(const std::string &json) { | |||
| // Start compiling.. | |||
| auto res = SendRequest(kFormat); | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2020 Huawei Technologies Co., Ltd | |||
| * Copyright 2020-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -52,6 +52,9 @@ class KernelBuildClient { | |||
| public: | |||
| // Send Finish request to server | |||
| constexpr inline static auto kFinish = "FINISH"; | |||
| constexpr inline static auto kAkgStart = "AKG/START"; | |||
| constexpr inline static auto kAkgData = "AKG/DATA"; | |||
| constexpr inline static auto kAkgWait = "AKG/WAIT"; | |||
| // Receive the response from server | |||
| constexpr inline static auto kAck = "ACK"; | |||
| constexpr inline static auto kErr = "ERR"; | |||
| @@ -124,6 +127,11 @@ class KernelBuildClient { | |||
| return res; | |||
| } | |||
| // Run AKG building. | |||
| bool AkgStart(int process_num, int wait_time); | |||
| bool AkgSendData(const std::vector<std::string> &jsons); | |||
| bool AkgWait(); | |||
| protected: | |||
| KernelBuildClient() : init_(false), dp_(std::make_shared<DuplexPipe>()) {} | |||
| virtual ~KernelBuildClient() = default; | |||
| @@ -197,9 +205,6 @@ class AscendKernelBuildClient : public KernelBuildClient { | |||
| constexpr inline static auto kTbeStart = "TBE/START"; | |||
| constexpr inline static auto kTbeWait = "TBE/WAIT"; | |||
| constexpr inline static auto kTbeReset = "TBE/RESET"; | |||
| constexpr inline static auto kAkgStart = "AKG/START"; | |||
| constexpr inline static auto kAkgData = "AKG/DATA"; | |||
| constexpr inline static auto kAkgWait = "AKG/WAIT"; | |||
| constexpr inline static auto kTbeTune = "TBE/TUNE"; | |||
| // Send server info. query to server | |||
| @@ -227,12 +232,6 @@ class AscendKernelBuildClient : public KernelBuildClient { | |||
| bool TbeWait(int *task_id, std::string *task_result, std::string *pre_build_result); | |||
| void TbeReset(); | |||
| // Run AKG building. | |||
| bool AkgStart(int process_num, int wait_time); | |||
| bool AkgSendData(const std::vector<std::string> &jsons); | |||
| bool AkgWait(); | |||
| bool AkgCompileSingle(const std::string json); | |||
| AscendKernelBuildClient(const AscendKernelBuildClient &) = delete; | |||
| AscendKernelBuildClient &operator=(const AscendKernelBuildClient &) = delete; | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2019 Huawei Technologies Co., Ltd | |||
| * Copyright 2019-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -95,7 +95,8 @@ static bool KernelBuildParallelCompile(const std::vector<CNodePtr> &kernels) { | |||
| } | |||
| } | |||
| bool tbe_ret = kernel::TbeOpParallelBuild(tbe_nodes); | |||
| bool akg_ret = kernel::AkgAscendKernelParallelBuild(akg_nodes); | |||
| kernel::AkgAscendKernelBuilder akg_ascend_kernel_builder; | |||
| bool akg_ret = akg_ascend_kernel_builder.AkgKernelParallelBuild(akg_nodes); | |||
| auto bin_map = kernel::tbe::KernelMeta::GetInstance(); | |||
| (void)bin_map->ReadIndex(kernel::kCceKernelMeta); | |||
| for (const auto &anf_node : other_nodes) { | |||
| @@ -1,5 +1,5 @@ | |||
| /** | |||
| * Copyright 2019 Huawei Technologies Co., Ltd | |||
| * Copyright 2019-2021 Huawei Technologies Co., Ltd | |||
| * | |||
| * Licensed under the Apache License, Version 2.0 (the "License"); | |||
| * you may not use this file except in compliance with the License. | |||
| @@ -33,6 +33,7 @@ void GpuBuild(const KernelGraphPtr &kernel_graph) { | |||
| MS_EXCEPTION_IF_NULL(kernel_graph); | |||
| bool already_check_nvcc = false; | |||
| auto kernels = kernel_graph->execution_order(); | |||
| std::vector<AnfNodePtr> akg_nodes; | |||
| for (const auto &kernel : kernels) { | |||
| std::string kernel_name = session::AnfRuntimeAlgorithm::GetCNodeName(kernel); | |||
| if (kernel_name == prim::kPrimTupleGetItem->name() || kernel_name == prim::kPrimMakeTuple->name() || | |||
| @@ -54,11 +55,7 @@ void GpuBuild(const KernelGraphPtr &kernel_graph) { | |||
| "cuda library(eg. /usr/local/cuda)."; | |||
| } | |||
| } | |||
| auto gpu_kernel_ptr = kernel::AkgGpuKernelBuild(kernel); | |||
| if (!gpu_kernel_ptr) { | |||
| MS_LOG(EXCEPTION) << "Build akg kernel op[" << kernel->fullname_with_scope() << "] failed"; | |||
| } | |||
| session::AnfRuntimeAlgorithm::SetKernelMod(gpu_kernel_ptr, kernel.get()); | |||
| akg_nodes.push_back(kernel); | |||
| } else { | |||
| auto gpu_kernel_ptr = kernel::GpuKernelFactory::GetInstance().Create(kernel_name, kernel); | |||
| if (!gpu_kernel_ptr) { | |||
| @@ -72,6 +69,21 @@ void GpuBuild(const KernelGraphPtr &kernel_graph) { | |||
| session::AnfRuntimeAlgorithm::SetKernelMod((kernel::KernelModPtr)gpu_kernel_ptr, kernel.get()); | |||
| } | |||
| } | |||
| struct timeval start_time, end_time; | |||
| (void)gettimeofday(&start_time, nullptr); | |||
| kernel::AkgGpuKernelBuilder akg_gpu_kernel_builder; | |||
| bool akg_ret = akg_gpu_kernel_builder.AkgKernelParallelBuild(akg_nodes); | |||
| if (!akg_ret) { | |||
| MS_LOG(ERROR) << "Akg-Kernel Parallel Building in GPU fail."; | |||
| } | |||
| (void)gettimeofday(&end_time, nullptr); | |||
| const uint64_t kUSecondInSecond = 1000000; | |||
| uint64_t cost = kUSecondInSecond * static_cast<uint64_t>(end_time.tv_sec - start_time.tv_sec); | |||
| cost += static_cast<uint64_t>(end_time.tv_usec - start_time.tv_usec); | |||
| MS_LOG(INFO) << "Akg GPU KernelBuild run in " << PRIu64 << " us " << cost; | |||
| } | |||
| } // namespace gpu | |||
| } // namespace device | |||