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 2021/02/03 01:26:58 UTC

[GitHub] [tvm] areusch opened a new pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

areusch opened a new pull request #7398:
URL: https://github.com/apache/tvm/pull/7398


   This PR is a follow-on to #7002 and continues its work to support BYOC functions with the `llvm` TVM target with the C runtime. Though the BYOC flow is supported with the TVM C++ runtime, this support depends heavily on `dlsym` and shared library support, neither of which are required to use the TVM C runtime.
   
   This PR can roughly be thought of in these parts:
    - Build a corresponding `runtime.CreateLLVMCrtMetadataModule` PackedFunc to the `runtime.CreateCSourceCrtMetadataModule`. Also add `crt` here to distinguish between these metadata modules (which are intended to be compiled) and runtime::MetadataModule (which is intended to be directly loaded into the TVM compiler c++ runtime)
    - Move FuncRegistry and TVMSystemLibEntryPoint LLVM generator into a new LLVMModule instance. Change function registry to declare extern functions rather than using internal linkage.
    - Allow `tvm.micro.build_static_runtime` to accept object files from the LLVM backend. This temporarily solution allows us to build `llvm`-targeted code using microTVM.
   
   


----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       @areusch , few clarfications : 
   
   1. The MetadataModule of type "metadata" is a another module that get exported using ProcessModuleBlob in c++ runtime. Therefore, I would not agree to  "MetadataModule is needed to export Modules ". For e.g., currently ARM Ethos-N that uses ProcessModuleBlob does not require MetadataModule of type "metadata" as it does not need any constants to be loaded into it in the init process. Having said that, what I agree with you is that ProcessModuleBlob should not be the way to do things in C runtime -- more specifically in uTVM environments. So you can think of  MetadataModule of type "metadata" as a peer to all runtime modules that gets "packed" to use ProcessModuleBlob in the runtime.
   
   2. Similarly, I would not agree "target_host parses binary configuration data in MetadataModule and then configures any runtime libraries or accelerators", because MetadataModule of type "metadata" only have the constants and other binaries go inside devc.c/o independently of MetadataModule of type "metadata". 
   
   I agree that " C runtime does not want to require any startup procedure that involves parsing binary blobs using RAM--instead, this must happen at code-generation time.".
   
   "This means that we need to do a better job here of organizing Module according to the DLContext which will be executing those modules." -- This I agree. First thing is we need to move away from the assumption that BYOC flows always generate non-DSO-exportable runtime.Modules that is not currently true for our ongoing ARM Ethos-U work. We will be producing a DSOExportable runtime.Module.  One side note to this, we might want to integrate the BYOC (possiblly not call it BYOC :) ) in the compilation pathway as it offers IRModule --> runtime.Module conversion generally and TVM's internal module compilation is "a" way of producing it. I have made a similiar comment here : https://github.com/apache/tvm/pull/7428
   
   So towards that goal, I think we need a mechanism to identify the DLContext without relying on their type being "c" or "llvm" as they could possible be coming from a BYOC flow. There should be a primary key in the runtime.Module to say its DLContext, IMHO.




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       yeah that seems right. I think we've come to similar conclusions. I think there are a couple of larger issues to be resolved here:
   1. When targeting the C runtime, an entirely different strategy than MetadataModule is needed to export Modules which don't run on the target_host CPU. This isn't well-defined yet. [Model Library Format RFC](https://discuss.tvm.apache.org/t/rfc-tvm-model-library-format/9121) also touches on this, but doesn't resolve it.
   2. Related to resolving this conundrum: the model of the C++ runtime is that some library running on target_host parses binary configuration data in MetadataModule and then configures any runtime libraries or accelerators. This is generally used to implement BYOC. C runtime does not want to require any startup procedure that involves parsing binary blobs using RAM--instead, this must happen at code-generation time. This means that we need to do a better job here of organizing Module according to the DLContext which will be executing those modules.
   
   Here we've mostly fixed problem 1. In future PR and RFC, we can work to address problem 2. Let me know if that makes sense to you!




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: tests/python/unittest/test_crt.py
##########
@@ -106,6 +105,20 @@ def test_compile_runtime():
         assert (C_data.asnumpy() == np.array([6, 7])).all()
 
 
+@tvm.testing.requires_micro
+def test_compile_runtime_llvm():
+    """Test targeting the on-device runtime with the llvm backend."""
+    global TARGET
+    old_target = TARGET
+    try:
+        TARGET = tvm.target.Target("llvm " + str(TARGET)[2:])

Review comment:
       Not very clear to me what is going on here ?
   I would imagine we would want to test both llvm and c targets, right ?
   May I ask why is the need for try-finally 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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       runtime::MetadataModuleCreate() --> is non-DSO-exportable it gets packed into to devc.{o/c} and I think we should only need this if we have modules that require constant loading. This one is of type "metadata". Im talking about the one that is created in line 96.
   
   I mean DSOExportable modules could still use constant loading -- though I think its not very useful since we have linked params now. Only remaining place is the example c byoc codegen that uses the constant loading process as a demo to all BYOC codegens.
   
   Thus, if none of the modules does not require constant loading -- that is indicated by arrays.empty() -- we dont need to create the non-DSOExportable metadata module.
   
   See the diagram here : https://discuss.tvm.apache.org/t/csourcemetadata-module-a-csourcemodule-to-hold-function-registry-for-utvm/8554.
   
   




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       runtime::MetadataModuleCreate() --> is non-DSO-exportable it gets packed into to devc.{o/c} and I think we should only need this if we have modules that require constant loading. This one is of type "metadata". Im talking about the one that is created in line 96.
   
   I mean DSOExportable modules could still use constant loading -- though I think its not very useful since we have linked params now. Only remaining place is the example c byoc codegen that uses the constant loading process as a demo to all BYOC codegens -- I think.
   
   Thus, if none of the modules does not require constant loading -- that is indicated by arrays.empty() -- we dont need to create the non-DSOExportable metadata module.
   
   See the diagram here : https://discuss.tvm.apache.org/t/csourcemetadata-module-a-csourcemodule-to-hold-function-registry-for-utvm/8554.
   
   Does that make sense ?
   
   




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       I think target_module should not be a binary module that gets imported inside non-DSOExportable metadata module.
   I think csource_modules --> dso_modules and binary_modules --> non_dso_modules would be better. Moreover,  non-DSOExportable metadata module should not be created if non of the modules need constant loading.
   
   TL;DR
   This one is little complex to explain. Let me use some definitions first (and feel free to change the names as you see fit).
   
   [Defn.1] binary module : a runtime.Module that is not DSOExportable (not of type "c" or "llvm").
   [Defn.2][NCL (needs constant loading)  module : a module that requires constant loading in the init process.
   
   As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen). 
   
   So technically any module that implements get_func_names could be imported under the DSOExportable metadata module (llvm -- the one this PR introduces or c -- the one I introduced before). However, currently the only ones that get imported inside DSOExportable metadata module are modules that are !("binary" or "NCL"). So I had a hard time finding the name and end-up calling them binary modules.
   
   
   
   
   
   




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       I think target_module should not be a binary module that gets wrapped inside non-DSOExportable metadata module.
   I think csource_modules --> dso_modules and binary_modules --> non_dso_modules would be better. Moreover,  non-DSOExportable metadata module should not be created if non of the modules need constant loading.
   
   TL;DR
   This one is little complex to explain. Let me use some definitions first (and feel free to change the names as you see fit).
   
   [Defn.1] binary module : a runtime.Module that is not DSOExportable (not of type "c" or "llvm").
   [Defn.2][NCL (needs constant loading)  module : a module that requires constant loading in the init process.
   
   As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen). 
   
   So technically any module that implements get_func_names could be imported under the DSOExportable metadata module (llvm -- the one this PR introduces or c -- the one I introduced before). However, currently the only ones that get imported inside DSOExportable metadata module are modules that are !("binary" or "NCL"). So I had a hard time finding the name and end-up calling them binary modules.
   
   
   
   
   
   




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       I think target_module should not be a binary module that gets wrapped inside non-DSOExportable metadata module.
   I think csource_modules --> dso_modules and binary_modules --> non_dso_modules would be better. Moreover,  non-DSOExportable metadata module should not be created if non of the modules need constant loading.
   
   TL;DR
   This one is little complex to explain. Let me use some definitions first (and feel free to change the names as you see fit).
   
   [Defn.1] binary module : a runtime.Module that is not DSOExportable (not of type "c" or "llvm").
   [Defn.2][NCL (needs constant loading)  module : a module that requires constant loading in the init process.
   
   As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen). 
   
   So technically any module that implements get_func_names could be imported under the DSOExportable metadata module (llvm -- the one this PR introduces or c -- the one I introduced before). However, currently the only ones that get imported inside DSOExportable metadata module are modules that are !("binary" or "NCL"). So I had a hard time finding the name and end-up calling them binary modules.
   
   
   
   
   
   

##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       I think target_module should not be a binary module that gets imported inside non-DSOExportable metadata module.
   I think csource_modules --> dso_modules and binary_modules --> non_dso_modules would be better. Moreover,  non-DSOExportable metadata module should not be created if non of the modules need constant loading.
   
   TL;DR
   This one is little complex to explain. Let me use some definitions first (and feel free to change the names as you see fit).
   
   [Defn.1] binary module : a runtime.Module that is not DSOExportable (not of type "c" or "llvm").
   [Defn.2][NCL (needs constant loading)  module : a module that requires constant loading in the init process.
   
   As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen). 
   
   So technically any module that implements get_func_names could be imported under the DSOExportable metadata module (llvm -- the one this PR introduces or c -- the one I introduced before). However, currently the only ones that get imported inside DSOExportable metadata module are modules that are !("binary" or "NCL"). So I had a hard time finding the name and end-up calling them binary modules.
   
   
   
   
   
   

##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       runtime::MetadataModuleCreate() --> is non-DSO-exportable it gets packed into to devc.{o/c} and I think we should only need this if we have modules that require constant loading. This one is of type "metadata". Im talking about the one that is created in line 96.
   
   I mean DSOExportable modules could still use constant loading -- though I think its not very useful since we have linked params now. Only remaining place is the example c byoc codegen that uses the constant loading process as a demo to all BYOC codegens.
   
   Thus, if none of the modules does not require constant loading -- that is indicated by arrays.empty() -- we dont need to create the non-DSOExportable metadata module.
   
   See the diagram here : https://discuss.tvm.apache.org/t/csourcemetadata-module-a-csourcemodule-to-hold-function-registry-for-utvm/8554.
   
   

##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       runtime::MetadataModuleCreate() --> is non-DSO-exportable it gets packed into to devc.{o/c} and I think we should only need this if we have modules that require constant loading. This one is of type "metadata". Im talking about the one that is created in line 96.
   
   I mean DSOExportable modules could still use constant loading -- though I think its not very useful since we have linked params now. Only remaining place is the example c byoc codegen that uses the constant loading process as a demo to all BYOC codegens.
   
   Thus, if none of the modules does not require constant loading -- that is indicated by arrays.empty() -- we dont need to create the non-DSOExportable metadata module.
   
   See the diagram here : https://discuss.tvm.apache.org/t/csourcemetadata-module-a-csourcemodule-to-hold-function-registry-for-utvm/8554.
   
   Does that make sense ?
   
   

##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       runtime::MetadataModuleCreate() --> is non-DSO-exportable it gets packed into to devc.{o/c} and I think we should only need this if we have modules that require constant loading. This one is of type "metadata". Im talking about the one that is created in line 96.
   
   I mean DSOExportable modules could still use constant loading -- though I think its not very useful since we have linked params now. Only remaining place is the example c byoc codegen that uses the constant loading process as a demo to all BYOC codegens -- I think.
   
   Thus, if none of the modules does not require constant loading -- that is indicated by arrays.empty() -- we dont need to create the non-DSOExportable metadata module.
   
   See the diagram here : https://discuss.tvm.apache.org/t/csourcemetadata-module-a-csourcemodule-to-hold-function-registry-for-utvm/8554.
   
   Does that make sense ?
   
   




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       >  Therefore, I would not agree to "MetadataModule is needed to export Modules ". For e.g., currently ARM Ethos-N that uses ProcessModuleBlob does not require MetadataModule of type "metadata" as it does not need any constants to be loaded into it in the init process.
   
   sure, but it depends on ProcessModuleBlob calling `runtime.module.loadbinary_ethos-n`, right? This is the part I think of most concern going forward--for µTVM, and I think we could say more broadly for CRT (no reason to support two flows in one runtime) we want to make it possible to do as much of this logic at compile time to produce a firmware binary that requires minimal startup procedures.
   
   >So you can think of MetadataModule of type "metadata" as a peer to all runtime modules that gets "packed" to use ProcessModuleBlob in the runtime.
   
   this makes sense to me, but I'd further qualify it to "all non-DSO-exportable runtime modules," at least for now.
   
   > Similarly, I would not agree "target_host parses binary configuration data in MetadataModule and then configures any runtime libraries or accelerators", because MetadataModule of type "metadata" only have the constants and other binaries go inside devc.c/o independently of MetadataModule of type "metadata".
   
   You're right--I was getting this confused with `codegen_blob` since the FuncRegistry logic doesn't have an explicit home in C++ land.
   
   > So towards that goal, I think we need a mechanism to identify the DLContext without relying on their type being "c" or "llvm" as they could possiblly be coming from a BYOC flow. There should be a primary key in the runtime.Module to say its DLContext, IMHO.
   
   I agree with a few clarifications. Perhaps better to discuss on the forum. The main points are: all BYOC are `ext_dev`, and perhaps some SoC may have a set of identical accelerators, some configured with one program and some with another. We probably should just consider how to support that case now rather than have to refactor that later--I think the difference is minor and amounts to coming up with another name for `runtime::Module` which indicates a) which `device_type` or BYOC accelerator the module is to run on and b) if multiple such things exist, a sort of "program name" for the particular accelerator program being generated.




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       runtime::MetadataModuleCreate() --> is non-DSO-exportable it gets packed into to devc.{o/c} and I think we should only need this if we have modules that require constant loading. This one is of type "metadata". Im talking about the one that is created in line 96.
   
   I mean DSOExportable modules could still use constant loading -- though I think its not very useful since we have linked params now. Only remaining place is the example c byoc codegen that uses the constant loading process as a demo to all BYOC codegens.
   
   Thus, if none of the modules does not require constant loading -- that is indicated by arrays.empty() -- we dont need to create the non-DSOExportable metadata module.
   
   See the diagram here : https://discuss.tvm.apache.org/t/csourcemetadata-module-a-csourcemodule-to-hold-function-registry-for-utvm/8554.
   
   Does that make sense ?
   
   




----------------------------------------------------------------
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 edited a comment on pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

Posted by GitBox <gi...@apache.org>.
areusch edited a comment on pull request #7398:
URL: https://github.com/apache/tvm/pull/7398#issuecomment-778495460


   @manupa-arm @leandron @tqchen @jroesch @gromero @tom-gall @MatthewARM @mshawcroft @u99127 @comaniac @zhiics 


----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       > As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen).
   Do you mean "(I.e. binary module could use constant loading..." here?
   
   okay I think my point of confusion here (and I think it's related to lack of experience with BYOC) is: what makes metadata module non-DSO-exportable?




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: python/tvm/micro/compiler.py
##########
@@ -81,6 +81,10 @@ def _target_from_sources(cls, sources):
         target_strs = set()
 
         for obj in sources:
+            print("read", obj)

Review comment:
       remove?




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       >  Therefore, I would not agree to "MetadataModule is needed to export Modules ". For e.g., currently ARM Ethos-N that uses ProcessModuleBlob does not require MetadataModule of type "metadata" as it does not need any constants to be loaded into it in the init process.
   
   sure, but it depends on ProcessModuleBlob calling `runtime.module.loadbinary_ethos-n`, right? This is the part I think of most concern going forward--for µTVM, and I think we could say more broadly for CRT (no reason to support two flows in one runtime) we want to make it possible to do as much of this logic at compile time to produce a firmware binary that requires minimal startup procedures.
   
   >So you can think of MetadataModule of type "metadata" as a peer to all runtime modules that gets "packed" to use ProcessModuleBlob in the runtime.
   
   this makes sense to me, but I'd further qualify it to "all non-DSO-exportable runtime modules," at least for now.
   
   > Similarly, I would not agree "target_host parses binary configuration data in MetadataModule and then configures any runtime libraries or accelerators", because MetadataModule of type "metadata" only have the constants and other binaries go inside devc.c/o independently of MetadataModule of type "metadata".
   
   You're right--I was getting this confused with `codegen_blob` since the FuncRegistry logic doesn't have an explicit home in C++ land.
   
   > So towards that goal, I think we need a mechanism to identify the DLContext without relying on their type being "c" or "llvm" as they could possiblly be coming from a BYOC flow. There should be a primary key in the runtime.Module to say its DLContext, IMHO.
   
   I agree with a few clarifications. Perhaps better to discuss on the forum. The main points are: all BYOC are `ext_dev`, and perhaps may have a set of identical accelerators, some configured with one program and some with another. We probably should just consider how to support that case now rather than have to refactor that later--I think the difference is minor and amounts to coming up with another name for `runtime::Module` which indicates a) which `device_type` or BYOC accelerator the module is to run on and b) if multiple such things exist, a sort of "program name" for the particular accelerator program being generated.




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       @areusch , few clarfications : 
   
   1. The MetadataModule of type "metadata" is a another module that get loaded using ProcessModuleBlob in c++ runtime. Therefore, I would not agree to  "MetadataModule is needed to export Modules ". For e.g., currently ARM Ethos-N that uses ProcessModuleBlob does not require MetadataModule of type "metadata" as it does not need any constants to be loaded into it in the init process. Having said that, what I agree with you is that ProcessModuleBlob should not be the way to do things in C runtime -- more specifically in uTVM environments. So you can think of  MetadataModule of type "metadata" as a peer to all runtime modules that gets "packed" to use ProcessModuleBlob in the runtime.
   
   2. Similarly, I would not agree "target_host parses binary configuration data in MetadataModule and then configures any runtime libraries or accelerators", because MetadataModule of type "metadata" only have the constants and other binaries go inside devc.c/o independently of MetadataModule of type "metadata". 
   
   I agree that " C runtime does not want to require any startup procedure that involves parsing binary blobs using RAM--instead, this must happen at code-generation time.".
   
   "This means that we need to do a better job here of organizing Module according to the DLContext which will be executing those modules." -- This I agree. First thing is we need to move away from the assumption that BYOC flows always generate non-DSO-exportable runtime.Modules that is not currently true for our ongoing ARM Ethos-U work. We will be producing a DSOExportable runtime.Module.  One side note to this, we might want to integrate the BYOC (possiblly not call it BYOC :) ) in the compilation pathway as it offers IRModule --> runtime.Module conversion generally and TVM's internal module compilation is "a" way of producing it. I have made a similiar comment here : https://github.com/apache/tvm/pull/7428
   
   So towards that goal, I think we need a mechanism to identify the DLContext without relying on their type being "c" or "llvm" as they could possiblly be coming from a BYOC flow. There should be a primary key in the runtime.Module to say its DLContext, IMHO.




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: python/tvm/micro/compiler.py
##########
@@ -81,6 +81,10 @@ def _target_from_sources(cls, sources):
         target_strs = set()
 
         for obj in sources:
+            print("read", obj)

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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       Yes, this seems right :)
   Let me just tighten the point  5
   
   "5.)  Then, any modules which are: a) non-DSO-exportable OR b) (not defining any const_vars OR result of get_const_vars() is empty() ) are imported into the resulting (here resulting means at least one of the two conditions is met) metadata module."
   
   Summary is 5b is covered 5a and metadata module of type "metadata" should only be created if one of the above condition is met. 
   Moreover, c runtime does not support ProcessModuleBlob(Im talking about : src/runtime/library_module.cc) anyway -- does it ? -- that could be used as a blanket assertion to check whether all modules are !5a and !5b if the target says c runtime ?
   
   
   
   
   




----------------------------------------------------------------
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 pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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


   @manupa-arm @leandron @tqchen @jroesch @gromero @tom-gall @MatthewARM @mshawcroft @u99127 


----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       > As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen).
   
   Do you mean "(I.e. binary module could use constant loading..." here?
   
   okay I think my point of confusion here (and I think it's related to lack of experience with BYOC) is: what makes metadata module non-DSO-exportable?




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: tests/python/unittest/test_crt.py
##########
@@ -106,6 +105,20 @@ def test_compile_runtime():
         assert (C_data.asnumpy() == np.array([6, 7])).all()
 
 
+@tvm.testing.requires_micro
+def test_compile_runtime_llvm():
+    """Test targeting the on-device runtime with the llvm backend."""
+    global TARGET
+    old_target = TARGET
+    try:
+        TARGET = tvm.target.Target("llvm " + str(TARGET)[2:])

Review comment:
       clarified a bit--i'm being lazy and reusing test_compile_runtime




----------------------------------------------------------------
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] tmoreau89 commented on pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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


   Thank you @areusch @zhiics @manupa-arm the PR has been merged.


----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       okay so let me summarize my understanding:
   1. at the end of the day, we want to produce a single module that imports all the others
   2. This function gets the Module that implements target_host (`target_module`) plus a list of all the BYOC modules (`ext_modules`).
   3. when building for the C++ runtime, we have runtime::MetadataModule for this root module. We assume each Module in`ext_modules` contains exactly one Relay function whose name is `get_symbol()` and which may use ConstantNodes whose values must be initialized from `get_const_vars()`. `get_func_names()` is unused because `dlsym` is used to resolve functions in the C++ runtime.
   4. when building for the C runtime, we use either the CSourceMetadataModule from #7002 or LLVMCrtMetadataModule from this PR. These are both DSO-exportable. The CSourceMetadataModule queries function names from and imports all DSO-exportable modules which don't define any const_vars. Let's call these "CRT-exportable modules" since we don't have a good name for them.
   5. Then, any modules which are: a) non-DSO-exportable, b) not codegen'd by c or llvm backend, OR c) not defining any const_vars are imported into the resulting metadata module.
   
   Presumably then the LLVM module should work the same way as the C-source one, the only change needed being that target->kind->name == "llvm" should also admit DSO-exportable functions to csource_modules. 
   
   Then, I'd propose the following name changes:
   - csource_modules -> crt_exportable_modules
   - binary_modules -> non_crt_exportable_modules




----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       @areusch , few clarfications : 
   
   1. The MetadataModule of type "metadata" is a another module that get loaded using ProcessModuleBlob in c++ runtime. Therefore, I would not agree to  "MetadataModule is needed to export Modules ". For e.g., currently ARM Ethos-N that uses ProcessModuleBlob does not require MetadataModule of type "metadata" as it does not need any constants to be loaded into it in the init process. Having said that, what I agree with you is that ProcessModuleBlob should not be the way to do things in C runtime -- more specifically in uTVM environments. So you can think of  MetadataModule of type "metadata" as a peer to all runtime modules that gets "packed" to use ProcessModuleBlob in the runtime.
   
   2. Similarly, I would not agree "target_host parses binary configuration data in MetadataModule and then configures any runtime libraries or accelerators", because MetadataModule of type "metadata" only have the constants and other binaries go inside devc.c/o independently of MetadataModule of type "metadata". 
   
   I agree that " C runtime does not want to require any startup procedure that involves parsing binary blobs using RAM--instead, this must happen at code-generation time.".
   
   "This means that we need to do a better job here of organizing Module according to the DLContext which will be executing those modules." -- This I agree. First thing is we need to move away from the assumption that BYOC flows always generate non-DSO-exportable runtime.Modules that is not currently true for our ongoing ARM Ethos-U work. We will be producing a DSOExportable runtime.Module.  One side note to this, we might want to integrate the BYOC (possiblly not call it BYOC :) ) in the compilation pathway as it offers IRModule --> runtime.Module conversion generally and TVM's internal module compilation is "a" way of producing it. I have made a similiar comment here : https://github.com/apache/tvm/pull/7428
   
   So towards that goal, I think we need a mechanism to identify the DLContext without relying on their type being "c" or "llvm" as they could possible be coming from a BYOC flow. There should be a primary key in the runtime.Module to say its DLContext, IMHO.




----------------------------------------------------------------
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] trevor-m commented on pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

Posted by GitBox <gi...@apache.org>.
trevor-m commented on pull request #7398:
URL: https://github.com/apache/tvm/pull/7398#issuecomment-789282373


   FYI, this appears to break TVM builds with USE_LLVM=OFF. Will try to come up with a fix


----------------------------------------------------------------
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] tmoreau89 merged pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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


   


----------------------------------------------------------------
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 pull request #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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


   @zhiics merge?


----------------------------------------------------------------
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 #7398: [CRT] Create C-runtime-style metadata module for llvm builds

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



##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       > As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen).
   Do you mean "(I.e. binary module could use constant loading..." here?
   
   okay I think my point of confusion here (and I think it's related to lack of experience with BYOC) is: what makes metadata module non-DSO-exportable?

##########
File path: src/target/metadata_module.cc
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file metadata_module.cc
+ * \brief Defines functions that build MetadataModules for C++ and C runtimes.
+ */
+
+#include "metadata_module.h"
+
+#include "../runtime/meta_data.h"
+#include "llvm/llvm_module.h"
+#include "source/source_module.h"
+
+namespace tvm {
+namespace codegen {
+
+/*!
+ * \brief Create a metadata module wrapper. The helper is used by different
+ *        codegens, such as graph runtime codegen and the vm compiler.
+ *
+ * \param params The metadata for initialization of all 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,
+    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 (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();
+      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);
+    }
+  }
+
+  if (target.defined() &&
+      target->GetAttr<String>("runtime").value_or(String("")) == kTvmRuntimeCrt) {
+    if (target->kind->name == "c") {
+      csource_modules.push_back(target_module);
+      target_module = CreateCSourceCrtMetadataModule(csource_modules, target);
+    } else if (target->kind->name == "llvm") {
+      binary_modules.push_back(target_module);

Review comment:
       > As of today these two things are not strictly orthogonal. (i.e. DSOExportabe module could use constant loading -- though the only one is the example byoc c codegen).
   
   Do you mean "(I.e. binary module could use constant loading..." here?
   
   okay I think my point of confusion here (and I think it's related to lack of experience with BYOC) is: what makes metadata module non-DSO-exportable?




----------------------------------------------------------------
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