You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2020/11/30 19:37:48 UTC

[GitHub] [tvm] manupa-arm opened a new pull request #7002: Created CSourceMetaData module for model metadata

manupa-arm opened a new pull request #7002:
URL: https://github.com/apache/tvm/pull/7002


   * Currently, there is a MetaData module to capture constants
     conditionaly if the runtime modules implement const init
     PackedFuncs. However, this one relies on a load process
     in which the metadata is created on volatile memory that
     may be not usable in uTVM environments.
   * There is a need for model level metadata that is valid
     across all runtime modules such as the func registry
     when creating a system-lib.
   * This commit implements a CSoureMetaData module to hold
     func registry that collects function names from the
     runtime module and generates a c source file to be
     linked with final artifact.
   * Modified and added export_library for utvm
   
   Discuss : ?
   
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546196370



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       Yes, that is correct.
   
   We would only need the current MetadataModule (which is non-DSOExportable) if the runtime module's requires the constants to be copied in the init process -- which is something we dont want in bare-metal. In-fact if the non of external modules have constants extracted (e.g., ethos-n and ethos-u will be managing their own constants), we dont even want MetadataModule (which is non-DSOExportable) get imported to the module hierarchy.
   
   Moreover, the purpose of the c-source metadata module is to provide an global entity for all DSOExportable modules a place to hold global data of the model. Initially its the function registry which @areusch describes here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#issuecomment-737363661


   @areusch @tqchen @mbaret @zhiics
   Please take a look when you have a minute :). 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546197542



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);
+    } else {
+      binary_modules.push_back(mod);
+    }
   }
 
-  // Wrap the modules.
-  runtime::Module init_m = runtime::MetadataModuleCreate(params, sym_metadata);
-  init_m.Import(dso_module);
-  for (const auto& it : modules) {
-    init_m.Import(it);
+  if (target.defined()) {

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r545600389



##########
File path: src/relay/backend/contrib/codegen_c/codegen.cc
##########
@@ -259,18 +263,13 @@ class CSourceCodegen : public CSourceModuleCodegenBase {
     )op_macro";
 
     code_stream_ << operator_macro << "\n\n";
-
-    ICHECK(ref->IsInstance<FunctionNode>());
-    auto res = GenCFunc(Downcast<Function>(ref));
+    code_stream_ << std::get<3>(res);
     std::string code = code_stream_.str();
 
-    String sym = std::get<0>(res);
-    Array<String> variables = std::get<1>(res);
-
     // Create a CSource module
     const auto* pf = runtime::Registry::Get("runtime.CSourceModuleCreate");
     ICHECK(pf != nullptr) << "Cannot find csource module to create the external runtime module";
-    return (*pf)(code, "c", sym, variables);
+    return (*pf)(code, "c", Array<String>{func_name}, sym, variables);

Review comment:
       Used func_name as it is more clearer term and removed sym.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546094897



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,
+    "ldflags": ["-std=gnu++11"],

Review comment:
       Reverted them back to c++11 and made export library save the runtime modules to .c instead .cc

##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,
+    "ldflags": ["-std=gnu++11"],

Review comment:
       Reverted them back to c++11 and made export library save the runtime modules to .c instead of .cc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] areusch commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
areusch commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546162527



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       with C runtime on bare metal, we have no `dlsym`, so we must assemble a string-to-function pointer lookup ourselves. that's why we pass them to metadata module here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] zhiics commented on pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#issuecomment-749223654


   Thanks @manupa-arm @areusch 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] zhiics commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r534335252



##########
File path: src/relay/backend/build_module.cc
##########
@@ -493,18 +493,14 @@ class RelayBuildModule : public runtime::ModuleNode {
         // If we cannot decide the target is LLVM, we create an empty CSourceModule.
         // The code content is initialized with ";" to prevent complaining
         // from CSourceModuleNode::SaveToFile.
-        ret_.mod = tvm::codegen::CSourceModuleCreate(";", "");
+        ret_.mod = tvm::codegen::CSourceModuleCreate(";", "", Array<String>{});
       }
     } else {
       ret_.mod = tvm::build(lowered_funcs, target_host_);
     }
 
-    Array<tvm::runtime::Module> ext_mods = graph_codegen_->GetExternalModules();
-    // TODO(zhiics) We should be able to completely switch to MetadataModule no
-    // matter whether there are external modules or not.
-    if (!ext_mods.empty()) {

Review comment:
       I think this would duplicate the param in the .so and .params file. I had a plan to handle it but didn't have cycles.

##########
File path: src/relay/backend/contrib/codegen_c/codegen.cc
##########
@@ -215,20 +215,24 @@ class CodegenC : public MemoizedExprTranslator<std::vector<Output>>, public Code
 
 class CSourceCodegen : public CSourceModuleCodegenBase {
  public:
-  std::pair<std::string, Array<String>> GenCFunc(const Function& func) {
+  std::tuple<std::string, Array<String>, String, String> GenCFunc(const Function& func) {
     ICHECK(func.defined()) << "Input error: expect a Relay function.";
 
     // Record the external symbol for runtime lookup.
     auto sid = GetExtSymbol(func);
 
     CodegenC builder(sid);
     auto out = builder.VisitExpr(func->body);
-    code_stream_ << builder.JIT(out);
-
-    return {sid, builder.const_vars_};

Review comment:
       I think sid is the same to ext_func_id_. 

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -1145,11 +1145,9 @@ void VMCompiler::Codegen() {
   } else {
     // There is no function handled by TVM. We create a virtual main module
     // to make sure a DSO module will be also available.
-    exec_->lib = codegen::CSourceModuleCreate(";", "");
-  }
-  if (!ext_mods.empty()) {
-    exec_->lib = codegen::CreateMetadataModule(params_, exec_->lib, ext_mods);

Review comment:
       same here

##########
File path: src/relay/backend/contrib/dnnl/codegen.cc
##########
@@ -413,7 +413,8 @@ class DNNLModuleCodegen : public CSourceModuleCodegenBase {
     // Create a CSource module
     const auto* pf = runtime::Registry::Get("runtime.CSourceModuleCreate");
     ICHECK(pf != nullptr) << "Cannot find csource module to create the external runtime module";
-    return (*pf)(code, "c", sym, variables);
+    // TODO(@manupa-arm): pass the function names to enable system-lib creation
+    return (*pf)(code, "c", Array<String>{}, sym, variables);

Review comment:
       Do we need an array? it seems that a csourcemodule is only one function though.

##########
File path: src/relay/backend/contrib/codegen_c/codegen.cc
##########
@@ -259,18 +263,13 @@ class CSourceCodegen : public CSourceModuleCodegenBase {
     )op_macro";
 
     code_stream_ << operator_macro << "\n\n";
-
-    ICHECK(ref->IsInstance<FunctionNode>());
-    auto res = GenCFunc(Downcast<Function>(ref));
+    code_stream_ << std::get<3>(res);
     std::string code = code_stream_.str();
 
-    String sym = std::get<0>(res);
-    Array<String> variables = std::get<1>(res);
-
     // Create a CSource module
     const auto* pf = runtime::Registry::Get("runtime.CSourceModuleCreate");
     ICHECK(pf != nullptr) << "Cannot find csource module to create the external runtime module";
-    return (*pf)(code, "c", sym, variables);
+    return (*pf)(code, "c", Array<String>{func_name}, sym, variables);

Review comment:
       I think we can use sym directly. It should be the same to func_name




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] areusch commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
areusch commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r534351281



##########
File path: python/tvm/runtime/module.py
##########
@@ -268,6 +268,11 @@ def export_library(self, file_name, fcompile=None, addons=None, **kwargs):
             If fcompile has attribute object_format, will compile host library
             to that format. Otherwise, will use default format "o".
 
+        workspace_dir : str, optional
+            The name of the workspace dir to create intermediary

Review comment:
       the path to a directory used to create

##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,

Review comment:
       do we need to change to gnu? would prefer to stick with c++11

##########
File path: python/tvm/runtime/module.py
##########
@@ -268,6 +268,11 @@ def export_library(self, file_name, fcompile=None, addons=None, **kwargs):
             If fcompile has attribute object_format, will compile host library
             to that format. Otherwise, will use default format "o".
 
+        workspace_dir : str, optional
+            The name of the workspace dir to create intermediary
+            artifacts for the process exporting of the library.
+            If this is not provided a temporary dir will be created.
+
         kwargs : dict, optional
             Additional arguments passed to fcompile
         """

Review comment:
       update the docstring to describe the return type

##########
File path: python/tvm/runtime/module.py
##########
@@ -330,25 +341,25 @@ def export_library(self, file_name, fcompile=None, addons=None, **kwargs):
 
         if self.imported_modules:
             if enabled("llvm") and llvm_target_triple:
-                path_obj = temp.relpath("devc." + object_format)
+                path_obj = os.path.join(workspace_dir, f"devc.{object_format}")
                 m = _ffi_api.ModulePackImportsToLLVM(self, is_system_lib, llvm_target_triple)
                 m.save(path_obj)
                 files.append(path_obj)
             else:
-                path_cc = temp.relpath("devc.cc")
+                path_cc = os.path.join(workspace_dir, "devc.cc")
                 with open(path_cc, "w") as f:
                     f.write(_ffi_api.ModulePackImportsToC(self, is_system_lib))
                 files.append(path_cc)
 
-        if has_c_module:
+        if has_c_module and not file_name.endswith(".tar"):

Review comment:
       can you add a comment to explain when .tar would be present here?

##########
File path: src/relay/backend/contrib/codegen_c/codegen.cc
##########
@@ -215,20 +215,24 @@ class CodegenC : public MemoizedExprTranslator<std::vector<Output>>, public Code
 
 class CSourceCodegen : public CSourceModuleCodegenBase {
  public:
-  std::pair<std::string, Array<String>> GenCFunc(const Function& func) {
+  std::tuple<std::string, Array<String>, String, String> GenCFunc(const Function& func) {
     ICHECK(func.defined()) << "Input error: expect a Relay function.";
 
     // Record the external symbol for runtime lookup.
     auto sid = GetExtSymbol(func);
 
     CodegenC builder(sid);
     auto out = builder.VisitExpr(func->body);
-    code_stream_ << builder.JIT(out);
-
-    return {sid, builder.const_vars_};

Review comment:
       would also prefer to avoid sid here, since it's used in graph_runtime extensively as an int.

##########
File path: src/target/source/source_module.cc
##########
@@ -46,20 +48,25 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param modules All the modules that needs to be imported inside the metadata module(s).

Review comment:
       ext_modules, correct? can you also add a line for target_module?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#issuecomment-747900398


   @tqchen @areusch @zhiics , I ve addressed all the comments and aligned with what we discussed in the RFC.
   Can you have another look when you have time? If its OK, please approve explicitly. Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] zhiics commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546161079



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       why do we need to extract csource_module? Originally the target module is a DSOModule. The other external modules are parallel to it like the following:
   
   ```
   MetadataModule
       -- DSO
       -- C source1
       -- C source2
       -- other type of modules
   ```
   Each sub module manages its own constants. Now we combine c source modules to the DSO. What's the advantage of it?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r545970811



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,
+    "ldflags": ["-std=gnu++11"],

Review comment:
       You may be right; Let me try this one.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546196370



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       Yes, that is correct.
   
   We would only need the current MetadataModule (which is non-DSOExportable) if the runtime module's requires the constants to be copied in the init process -- which is something we dont want in bare-metal. In-fact if the non of external modules have constants extracted (e.g., ethos-n and ethos-u will be managing their own constants -- truly with the current approach each sub module are copied with their constants from the MetadataModule by the MetadataModule), we dont even want MetadataModule (which is non-DSOExportable) get imported to the module hierarchy.
   
   Moreover, the purpose of the c-source metadata module is to provide an global entity for all DSOExportable modules a place to hold global data of the model. Initially its the function registry which @areusch describes here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544574734



##########
File path: python/tvm/runtime/module.py
##########
@@ -330,25 +341,25 @@ def export_library(self, file_name, fcompile=None, addons=None, **kwargs):
 
         if self.imported_modules:
             if enabled("llvm") and llvm_target_triple:
-                path_obj = temp.relpath("devc." + object_format)
+                path_obj = os.path.join(workspace_dir, f"devc.{object_format}")
                 m = _ffi_api.ModulePackImportsToLLVM(self, is_system_lib, llvm_target_triple)
                 m.save(path_obj)
                 files.append(path_obj)
             else:
-                path_cc = temp.relpath("devc.cc")
+                path_cc = os.path.join(workspace_dir, "devc.cc")
                 with open(path_cc, "w") as f:
                     f.write(_ffi_api.ModulePackImportsToC(self, is_system_lib))
                 files.append(path_cc)
 
-        if has_c_module:
+        if has_c_module and not file_name.endswith(".tar"):

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r545970593



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {

Review comment:
       Not sure I follow; 
   
   Irrespective of link-params, we would need the DSOModules to fall under c-source metadata module if the target is "c" and the runtime module does not require constant loading.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544579500



##########
File path: src/target/source/source_module.cc
##########
@@ -46,20 +48,25 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param modules All the modules that needs to be imported inside the metadata module(s).

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#issuecomment-749003721


   Ping @areusch @zhiics.
   Let me know if you have any more concerns, I ll be disappearing for holidays from tomorrow. 
   Thus, would like to address if there is anything today. Thanks! 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] zhiics merged pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
zhiics merged pull request #7002:
URL: https://github.com/apache/tvm/pull/7002


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544574591



##########
File path: python/tvm/runtime/module.py
##########
@@ -268,6 +268,11 @@ def export_library(self, file_name, fcompile=None, addons=None, **kwargs):
             If fcompile has attribute object_format, will compile host library
             to that format. Otherwise, will use default format "o".
 
+        workspace_dir : str, optional
+            The name of the workspace dir to create intermediary
+            artifacts for the process exporting of the library.
+            If this is not provided a temporary dir will be created.
+
         kwargs : dict, optional
             Additional arguments passed to fcompile
         """

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544574436



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,

Review comment:
       I think current implementation of linked-params uses hexadecimal notation that requires gnu++. This is partly because export_library saves c-source modules .cc.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] areusch commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
areusch commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r545944222



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,
+    "ldflags": ["-std=gnu++11"],

Review comment:
       this shouldn't be needed by hex floating point literals, right?

##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,

Review comment:
       hm, but the code generator produces C, and hex floating point literals are part of c11. I don't think any c++ code contains those.

##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {

Review comment:
       perhaps it is better to enable this based on --link-params?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546094532



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,

Review comment:
       Alright, I think the correct thing to do is make export library save the runtime modules as .c not .cc. Thus, did that and reverted these changes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544574436



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,

Review comment:
       I think current implementation of linked-params uses floating point notation that requires gnu++. This is partly because export_library saves c-source modules .cc.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544574951



##########
File path: src/relay/backend/contrib/codegen_c/codegen.cc
##########
@@ -215,20 +215,24 @@ class CodegenC : public MemoizedExprTranslator<std::vector<Output>>, public Code
 
 class CSourceCodegen : public CSourceModuleCodegenBase {
  public:
-  std::pair<std::string, Array<String>> GenCFunc(const Function& func) {
+  std::tuple<std::string, Array<String>, String, String> GenCFunc(const Function& func) {
     ICHECK(func.defined()) << "Input error: expect a Relay function.";
 
     // Record the external symbol for runtime lookup.
     auto sid = GetExtSymbol(func);
 
     CodegenC builder(sid);
     auto out = builder.VisitExpr(func->body);
-    code_stream_ << builder.JIT(out);
-
-    return {sid, builder.const_vars_};

Review comment:
       Removed sid and just used ext_func_id_




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] areusch commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
areusch commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546008379



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {

Review comment:
       oh, sorry you're right. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] zhiics commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546159480



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);
+    } else {
+      binary_modules.push_back(mod);
+    }
   }
 
-  // Wrap the modules.
-  runtime::Module init_m = runtime::MetadataModuleCreate(params, sym_metadata);
-  init_m.Import(dso_module);
-  for (const auto& it : modules) {
-    init_m.Import(it);
+  if (target.defined()) {

Review comment:
       this can if `arget.defined()` && `target->kind->name`

##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       why do we need to extract csource_module? Originally the target module is a DSOModule. The other external module are parallel to it like the following:
   
   ```
   MetadataModule
       -- DSO
       -- C source1
       -- C source2
       -- other type of modules
   ```
   Each sub module manages their own constants. Now we combine c source modules to the DSO. What's the advantage of it?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546196370



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       Yes, that is correct.
   
   We would only need the current MetadataModule (which is non-DSOExportable) if the runtime module's requires the constants to be copied in the init process -- which is something we dont want in bare-metal. In-fact if the non of external modules have constants extracted (e.g., ethos-n and ethos-u will be managing their own constants -- truly with the current approach each sub module are copied with their constants from the MetadataModule by the MetadataModule), we dont even want MetadataModule (which is non-DSOExportable) get imported to the module hierarchy.
   
   Moreover, the purpose of the c-source metadata module is to provide an global entity for all DSOExportable modules a place to hold global data of the model. Initially its the function registry which @areusch describes here. These metadata is accessed via static linkage and therefore does not require an explicit copy.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r544579729



##########
File path: python/tvm/runtime/module.py
##########
@@ -268,6 +268,11 @@ def export_library(self, file_name, fcompile=None, addons=None, **kwargs):
             If fcompile has attribute object_format, will compile host library
             to that format. Otherwise, will use default format "o".
 
+        workspace_dir : str, optional
+            The name of the workspace dir to create intermediary

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] areusch commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
areusch commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546124976



##########
File path: python/tvm/micro/build.py
##########
@@ -74,8 +74,8 @@ def path(self):
 
 _CRT_DEFAULT_OPTIONS = {
     "cflags": ["-std=c11"] + _COMMON_CFLAGS,
-    "ccflags": ["-std=c++11"] + _COMMON_CFLAGS,
-    "ldflags": ["-std=c++11"],
+    "ccflags": ["-std=gnu++11"] + _COMMON_CFLAGS,

Review comment:
       ah, I just realized that was what was going on here. I agree we shouldn't be generating c++ for the c backend.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r534348967



##########
File path: src/relay/backend/contrib/dnnl/codegen.cc
##########
@@ -413,7 +413,8 @@ class DNNLModuleCodegen : public CSourceModuleCodegenBase {
     // Create a CSource module
     const auto* pf = runtime::Registry::Get("runtime.CSourceModuleCreate");
     ICHECK(pf != nullptr) << "Cannot find csource module to create the external runtime module";
-    return (*pf)(code, "c", sym, variables);
+    // TODO(@manupa-arm): pass the function names to enable system-lib creation
+    return (*pf)(code, "c", Array<String>{}, sym, variables);

Review comment:
       When we target "c", in it can have more than one function which also uses the same CSourceModuleCreate interface. See codegen_c_host.cc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] manupa-arm commented on a change in pull request #7002: Created CSourceMetaData module for model metadata

Posted by GitBox <gi...@apache.org>.
manupa-arm commented on a change in pull request #7002:
URL: https://github.com/apache/tvm/pull/7002#discussion_r546196370



##########
File path: src/target/source/source_module.cc
##########
@@ -46,40 +48,68 @@ using runtime::SaveBinaryToFile;
  *        codegens, such as graph runtime codegen and the vm compiler.
  *
  * \param params The metadata for initialization of all modules.
- * \param dso_module The DSO module that contains TVM primitives.
- * \param modules The submodules that will be wrapped, e.g. CSource modules that
- *        contain vendor library calls or customized runtime modules.
- *
+ * \param target_module the internal module that is compiled by tvm.
+ * \param ext_modules The external modules that needs to be imported inside the metadata
+ * module(s).
+ * \param target The target that all the modules are compiled for
  * \return The created metadata module that manages initialization of metadata.
  */
 runtime::Module CreateMetadataModule(
     const std::unordered_map<std::string, runtime::NDArray>& params,
-    const runtime::Module& dso_module, const Array<runtime::Module>& modules) {
+    tvm::runtime::Module target_module, const Array<runtime::Module>& ext_modules, Target target) {
+  Array<tvm::runtime::Module> csource_modules;
+  Array<tvm::runtime::Module> binary_modules;
+
+  auto DSOExportable = [](tvm::runtime::Module& mod) {
+    return !std::strcmp(mod->type_key(), "llvm") || !std::strcmp(mod->type_key(), "c");
+  };
+
   // Wrap all submodules in the initialization wrapper.
   std::unordered_map<std::string, std::vector<std::string>> sym_metadata;
-  for (runtime::Module it : modules) {
-    auto pf_sym = it.GetFunction("get_symbol");
-    auto pf_var = it.GetFunction("get_const_vars");
+  for (tvm::runtime::Module mod : ext_modules) {
+    auto pf_sym = mod.GetFunction("get_symbol");
+    auto pf_var = mod.GetFunction("get_const_vars");
+    std::vector<std::string> arrays;
     if (pf_sym != nullptr && pf_var != nullptr) {
       String symbol = pf_sym();
       Array<String> variables = pf_var();
-      std::vector<std::string> arrays;
       for (size_t i = 0; i < variables.size(); i++) {
         arrays.push_back(variables[i].operator std::string());
       }
       ICHECK_EQ(sym_metadata.count(symbol), 0U) << "Found duplicated symbol: " << symbol;
       sym_metadata[symbol] = arrays;
     }
+    // We only need loading of serialized constant data
+    // if there are constants present and required by the
+    // runtime module to be initialized by the binary
+    // metadata module. If not rest of the modules are
+    // wrapped in c-source metadata module.
+
+    // TODO(@manupa-arm) : we should be able to use csource_metadata
+    // if the variables are empty when all the runtime modules implement get_func_names
+    if (arrays.empty() && DSOExportable(mod) && target->kind->name == "c") {
+      csource_modules.push_back(mod);

Review comment:
       Yes, that is correct.
   
   We would only need the current MetadataModule (which is non-DSOExportable) if the runtime module's requires the constants to be copied in the init process -- which is something we dont want in bare-metal. In-fact if the non of external modules have constants extracted (e.g., ethos-n and ethos-u will be managing their own constants -- truly with the current approach each sub module are copied with their constants from the MetadataModule by the MetadataModule), we dont even want MetadataModule (which is non-DSOExportable) get imported to the module hierarchy.
   
   Moreover, the purpose of the c-source metadata module is to provide an global entity for all DSOExportable modules a place to hold global data of the model. Initially its the function registry which @areusch describes here. These metadata is accessed via linkage and therefore does not require an explicit copy.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org