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/05/12 10:01:16 UTC

[GitHub] [tvm] Mousius opened a new pull request #8023: [AOT] Initial implementation of --no-typed-operators

Mousius opened a new pull request #8023:
URL: https://github.com/apache/tvm/pull/8023


   Based on the discussions in [the AOT embedded improvements RFC](https://discuss.tvm.apache.org/t/rfc-utvm-aot-optimisations-for-embedded-targets/9849), this adds a flag to the target which changes the internal operators to an unpacked API. The unpacked API spreads the input buffers across the operator function, for example:
   ```
   int32_t operator(void* arg0, void* arg1);
   ```
   As opposed to the traditional packed API:
   ```
   int32_t operator(void** args);
   ```
   Unaffected is the entrypoint function, which retains a packed API for compatibility with other parts of TVM. The entrypoint function is generated as part of the metadata as suggested by @tqchen so we can easily swap it for implementing `--micro-entrypoint`. 
   
   cc: @giuseros @mbaret @manupa-arm @areusch @tqchen 


-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -564,21 +582,21 @@ class AOTExecutorCodegen : public ExprVisitor {
 
  public:
   AOTExecutorCodegen(runtime::Module* mod, const TargetsMap& targets, Target target_host)
-      : mod_(mod), return_sid_() {
+      : mod_(mod), use_typed_operators_(true) {

Review comment:
       Took another pass over this morning and fixed up the other initialisations too, thanks for pointing this out :+1: 




-- 
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] Mousius commented on pull request #8023: [AOT] Initial implementation of --unpacked-api

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


   @areusch I think I've incorporated all the changes and CI is still green :tada: 


-- 
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] Mousius commented on pull request #8023: [AOT] Initial implementation of --unpacked-api

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


   @areusch I think I've incorporated all the changes and CI is still green :tada: 


-- 
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] giuseros commented on a change in pull request #8023: [AOT] Initial implementation of --no-typed-operators

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -605,7 +625,8 @@ class AOTExecutorCodegen : public ExprVisitor {
     // Find the return sid
     return_sid_ = AotReturnSidVisitor(storage_device_map_).FindReturnSid(func);
     for (unsigned int output_index = 0; output_index < return_sid_.size(); output_index++) {
-      main_signature_.push_back(tir::Var(MakeString("output_", output_index), DataType::Handle()));
+      auto output_var = tir::Var("output", DataType::Handle());
+      main_signature_.push_back(output_var);

Review comment:
       Same here

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -588,10 +609,9 @@ class AOTExecutorCodegen : public ExprVisitor {
     auto pf = GetPackedFunc("relay.backend.GraphPlanMemory");
     storage_device_map_ = (*pf)(func);
 
-    int input_index = 0;
     for (auto input : func->params) {
       input_vars_.push_back(input);
-      main_signature_.push_back(tir::Var(MakeString("input_", input_index), DataType::Handle()));
+      main_signature_.push_back(tir::Var("input", DataType::Handle()));

Review comment:
       Why removing the indices? It's useful  when we read TIR to see the inputs/outputs numbered. 

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -137,10 +137,17 @@ class AOTExecutorCodegen : public ExprVisitor {
       // Pack the sid inside the TVMValue
       auto sid_array = te::Var(MakeString("sid_", sid, "_value"), DataType::Handle());
       auto sid_value = sids_table_[sid];
-      tvm::PrimExpr set_tensor =
-          tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
-                         {sid_array, 0, tir::builtin::kArrData, sid_value});
-      stmts_.push_back(tir::LetStmt(sid_array, StackAlloca("array", 1), tir::Evaluate(set_tensor)));
+
+      if (target_host_->GetAttr<Bool>("typed-operators").value_or(Bool(true))) {

Review comment:
       Why don't you create a flag `use_typed_operators_` ? 

##########
File path: tests/python/relay/aot/test_crt_aot.py
##########
@@ -43,7 +43,8 @@
 from aot_test_utils import *
 
 
-def test_conv_with_params():
+@pytest.mark.parametrize("target_options", ["", "--typed-operators=0"])

Review comment:
       I am worried about test explosion. Once you rebase and incorporate @manupa-arm changes (i.e., memory workspace feature) each test would run now 4 times [with/without workspace] x [with/without typed-operators]. Maybe you could select some tests where enabling this feature? Or maybe enable it by default, and add few tests without the feature?




-- 
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 #8023: [AOT] Initial implementation of --typed-operators

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -564,21 +582,21 @@ class AOTExecutorCodegen : public ExprVisitor {
 
  public:
   AOTExecutorCodegen(runtime::Module* mod, const TargetsMap& targets, Target target_host)
-      : mod_(mod), return_sid_() {
+      : mod_(mod), use_typed_operators_(true) {

Review comment:
       why do you double-initialize?

##########
File path: src/driver/driver_api.cc
##########
@@ -200,8 +200,15 @@ std::pair<IRModule, IRModule> SplitDevHostFuncs(IRModule mod_mixed, const Target
   mixed_pass_list.push_back(tir::transform::ThreadSync("warp"));
   mixed_pass_list.push_back(tir::transform::InferFragment());
   mixed_pass_list.push_back(tir::transform::LowerThreadAllreduce());
-  mixed_pass_list.push_back(tir::transform::MakePackedAPI(0));
+
+  if (target->GetAttr<Bool>("typed-operators").value_or(Bool(true))) {

Review comment:
       at the Target level, `--typed-operators` reads pretty generically and could imply something like "operator + is aware of the types of its arguments." but that's always true.
   
   i'd suggest two modifications:
   1. given we are abusing Target to hold runtime-specific information, let's choose a name for which the default 0 value preserves the existing behavior. `--typed-operators` defaults `true` here, but we can't document that properly (in `target_kind.cc`) since we are abusing Target.
   2. let's choose a more specific name, such as `--dltensor-only-function-signatures`. or alternatively, something that references "call_unpacked," maybe `--unpacked-api`? i don't know that `--typed-signatures` fully encapsulates the effect of the flag, since it not only removes `type_code` but also assumes there is a `DLTensor` for each argument.

##########
File path: src/tir/transforms/make_unpacked_api.cc
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 make_unpacked_api.cc Lower PrimFunc to a standard C function API.
+ */
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/analysis.h>
+#include <tvm/tir/buffer.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/expr.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "arg_binder.h"
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+PrimFunc MakeUnpackedAPI(PrimFunc&& func) {
+  auto global_symbol = func->GetAttr<String>(tvm::attr::kGlobalSymbol);
+  ICHECK(global_symbol) << "MakeUnpackedAPI: Expect PrimFunc to have the global_symbol attribute";
+
+  auto target = func->GetAttr<Target>(tvm::attr::kTarget);
+  ICHECK(target.defined()) << "MakeUnpackedAPI: Require the target attribute";
+
+  auto* func_ptr = func.CopyOnWrite();
+
+  // Setup device context
+  int target_device_type = target.value()->kind->device_type;
+  Integer device_type(target_device_type);
+  Integer device_id(0);
+  PrimExpr node = StringImm("default");
+  const Stmt nop = Evaluate(0);
+  std::vector<Stmt> device_init;
+
+  // Create arg to buffer binder
+  std::unordered_map<const VarNode*, PrimExpr> vmap;
+  ArgBinder binder(&vmap);
+
+  // Collect variables and buffers to map between
+  Array<Var> args;
+  std::vector<std::pair<Var, Var>> var_def;
+  std::vector<std::pair<Var, Buffer>> buffer_def;
+
+  for (int i = 0; i < static_cast<int>(func_ptr->params.size()); ++i) {
+    Var param = func_ptr->params[i];

Review comment:
       i'm not sure if we should ICHECK on the parameter type 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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: src/target/target_kind.cc
##########
@@ -219,6 +219,7 @@ TVM_REGISTER_TARGET_KIND("llvm", kDLCPU)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<String>("runtime")
     .add_attr_option<Bool>("link-params", Bool(false))
+    .add_attr_option<Bool>("unpacked-api", Bool(false))

Review comment:
       How come this leaked through between PR builds?

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -535,6 +551,8 @@ class AOTExecutorCodegen : public ExprVisitor {
   TargetsMap targets_;
   /*! \brief target host */
   Target target_host_;
+  /*! \brief untyped operators flag */

Review comment:
       Oh yeah, this comment is already out of date.




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       In this PR we're generating a packed function to wrap the AOT executor function, would it be worth emitting `tvm_module_main` rather than creating a new `tvm_entrypoint_name` ?




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

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



[GitHub] [tvm] giuseros commented on a change in pull request #8023: [AOT] Initial implementation of --typed-operators

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



##########
File path: src/target/source/source_module.cc
##########
@@ -192,17 +192,59 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
           << "}\n";
   }
 
+  void GenerateUntypedEntrypoint() {
+    code_ << "TVM_DLL int32_t " << ::tvm::runtime::symbol::tvm_run_func_prefix << "(";
+    int total_args = (metadata_->num_inputs + metadata_->num_outputs);
+    for (int i = 0; i < total_args; ++i) {
+      code_ << "arg" << i;
+      if (i + 1 != total_args) {
+        code_ << ",";
+      }
+    }
+    code_ << ");\n";
+    code_ << "static int32_t _tvm_entrypoint";
+    code_ << "(void* args, void* type_code, int num_args, void* out_value, void* "
+             "out_type_code, void* resource_handle) {\n";
+    code_ << "return " << ::tvm::runtime::symbol::tvm_run_func_prefix << "(";
+    for (int i = 0; i < metadata_->num_inputs; ++i) {
+      code_ << "((DLTensor*)(((TVMValue*)args)[" << i << "].v_handle))[0].data,";
+    }
+    for (int i = 0; i < metadata_->num_outputs; ++i) {
+      int j = metadata_->num_inputs + i;
+      code_ << "((DLTensor*)(((TVMValue*)args)[" << j << "].v_handle))[0].data";
+      if (i + 1 != metadata_->num_outputs) {
+        code_ << ",";
+      }
+    }
+    code_ << ");\n";
+    code_ << "}\n";
+  }
+
+  void GenerateTypedEntrypoint() {
+    code_ << "TVM_DLL int32_t " << ::tvm::runtime::symbol::tvm_run_func_prefix;
+    code_ << "(void* args, void* type_code, int num_args, void* out_value, void* "
+             "out_type_code, void* resource_handle);\n";
+    code_ << "static int32_t _tvm_entrypoint";
+    code_ << "(void* args, void* type_code, int num_args, void* out_value, void* "
+             "out_type_code, void* resource_handle) {\n";
+    code_ << "return " << ::tvm::runtime::symbol::tvm_run_func_prefix;
+    code_ << "(args, type_code, num_args, out_value, out_type_code, resource_handle);\n";
+    code_ << "}\n";
+  }
+
   void GenerateAOTDescriptor() {
     code_ << "#include \"tvm/runtime/crt/internal/aot_executor/aot_executor.h\"\n";
     code_ << "#include \"tvm/runtime/c_runtime_api.h\"\n";
     code_ << "#ifdef __cplusplus\n";
     code_ << "extern \"C\"\n";
     code_ << "#endif\n";
-    code_ << "TVM_DLL int32_t " << ::tvm::runtime::symbol::tvm_run_func_prefix;
-    code_ << "(void* args, void* type_code, int num_args, void* out_value, void* "
-             "out_type_code, void* resource_handle);\n";
+    if (target_->GetAttr<Bool>("typed-operators").value_or(Bool(true))) {
+      GenerateTypedEntrypoint();
+    } else {
+      GenerateUntypedEntrypoint();
+    }
     code_ << "const tvm_model_t network = {\n"
-          << "    .run_func = &" << ::tvm::runtime::symbol::tvm_run_func_prefix << ",\n"
+          << "    .run_func = &_tvm_entrypoint,\n"

Review comment:
       Maybe define a constant string and store it as a class member (e.g., `tvm_entrypoint_name`) , so we have only one place where this is defined. 




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: src/target/target_kind.cc
##########
@@ -219,6 +219,7 @@ TVM_REGISTER_TARGET_KIND("llvm", kDLCPU)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<String>("runtime")
     .add_attr_option<Bool>("link-params", Bool(false))
+    .add_attr_option<Bool>("unpacked-api", Bool(false))

Review comment:
       Yip, I'll remove the default for it, was just curious :smile_cat: 




-- 
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 #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: src/target/target_kind.cc
##########
@@ -219,6 +219,7 @@ TVM_REGISTER_TARGET_KIND("llvm", kDLCPU)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<String>("runtime")
     .add_attr_option<Bool>("link-params", Bool(false))
+    .add_attr_option<Bool>("unpacked-api", Bool(false))

Review comment:
       misconfiguration in some of the build scripts (they were recycling tuning logs from previous builds). however, it's the same effect as if the option was removed later on, and someone who never gave the option to their Target had tuned with it.




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

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



[GitHub] [tvm] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --no-typed-operators

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



##########
File path: tests/python/relay/aot/test_crt_aot.py
##########
@@ -43,7 +43,8 @@
 from aot_test_utils import *
 
 
-def test_conv_with_params():
+@pytest.mark.parametrize("target_options", ["", "--typed-operators=0"])

Review comment:
       I'm going to leave the smaller tests with all permutations as they're fast. The worrisome one is Mobilenet so I'll default that to having `typed-operators=0` and the memory tests?




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --typed-operators

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -564,21 +582,21 @@ class AOTExecutorCodegen : public ExprVisitor {
 
  public:
   AOTExecutorCodegen(runtime::Module* mod, const TargetsMap& targets, Target target_host)
-      : mod_(mod), return_sid_() {
+      : mod_(mod), use_typed_operators_(true) {

Review comment:
       There's no default constructor for `Bool`, so it tries to default initialise it and fails.




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --typed-operators

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



##########
File path: src/driver/driver_api.cc
##########
@@ -200,8 +200,15 @@ std::pair<IRModule, IRModule> SplitDevHostFuncs(IRModule mod_mixed, const Target
   mixed_pass_list.push_back(tir::transform::ThreadSync("warp"));
   mixed_pass_list.push_back(tir::transform::InferFragment());
   mixed_pass_list.push_back(tir::transform::LowerThreadAllreduce());
-  mixed_pass_list.push_back(tir::transform::MakePackedAPI(0));
+
+  if (target->GetAttr<Bool>("typed-operators").value_or(Bool(true))) {

Review comment:
       I replied on the RFC agreeing to this sentiment, my proposal is something like `--packed-functions` which defaults to running MakePackedAPI? Maybe `--packed-internal-functions` or `--packed-operator-functions`? 




-- 
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 #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       yeah that seems fine to me. what i was saying was that in the multiple-model-in-one-Module case, the convention from Module-based Model Library Format would be to use the model name as the function name here. However, I think multiple models could be supported by placing them in separate Modules. I don't know of any use cases that place both models in the same Module now.




-- 
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] Mousius commented on pull request #8023: [AOT] Initial implementation of --unpacked-api

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


   @areusch I think this is good to go, could you take another look? 


-- 
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 merged pull request #8023: [AOT] Initial implementation of --unpacked-api

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


   


-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: src/driver/driver_api.cc
##########
@@ -200,8 +200,15 @@ std::pair<IRModule, IRModule> SplitDevHostFuncs(IRModule mod_mixed, const Target
   mixed_pass_list.push_back(tir::transform::ThreadSync("warp"));
   mixed_pass_list.push_back(tir::transform::InferFragment());
   mixed_pass_list.push_back(tir::transform::LowerThreadAllreduce());
-  mixed_pass_list.push_back(tir::transform::MakePackedAPI(0));
+
+  if (target->GetAttr<Bool>("typed-operators").value_or(Bool(true))) {

Review comment:
       Updated to `--unpacked-api` based on RFC comments




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --no-typed-operators

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -588,10 +609,9 @@ class AOTExecutorCodegen : public ExprVisitor {
     auto pf = GetPackedFunc("relay.backend.GraphPlanMemory");
     storage_device_map_ = (*pf)(func);
 
-    int input_index = 0;
     for (auto input : func->params) {
       input_vars_.push_back(input);
-      main_signature_.push_back(tir::Var(MakeString("input_", input_index), DataType::Handle()));
+      main_signature_.push_back(tir::Var("input", DataType::Handle()));

Review comment:
       This was some minor boyscouting as code gen seems to already do this for us:
   ```
   TVM_DLL int32_t tvm__run_func(void* arg0, void* arg1, void* arg2, void* arg3, void* arg4) {
     void* input = arg0;
     void* output = arg1;
     void* output1 = arg2;
     void* output2 = arg3;
     void* output3 = arg4;
     (void)fused_add_1(input, output);
     (void)fused_add_1(output, output1);
     (void)fused_add_1(output1, output2);
     (void)fused_add_1(output2, output3);
     return 0;
   }
   ```
   
   Also, `input_index` wasn't incremented so you'd get output like this instead:
   ```
   TVM_DLL int32_t tvm__run_func(void* arg0, void* arg1, void* arg2, void* arg3) {
     void* input_0 = arg0;
     void* input_01 = arg1;
     void* input_02 = arg2;
     void* output_0 = arg3;
     (void)fused_concatenate_add_1(input_0, input_01, input_02, output_0);
     return 0;
   }
   ```




-- 
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 #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: tests/python/relay/aot/test_crt_aot.py
##########
@@ -44,7 +44,8 @@
 
 
 @pytest.mark.parametrize("use_calculated_workspaces", [True, False])
-def test_conv_with_params(use_calculated_workspaces):
+@pytest.mark.parametrize("target_options", ["", "--unpacked-api"])

Review comment:
       it might make sense to specify --unpacked-api=0 rather than the implicit default behavior here, so that tests explicitly cover the packed-api case

##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       let's at least update the comment here (and above, in the other constants) to explain the difference between `tvm_run_func_prefix`, `tvm_module_main`, and `tvm_entrypoint_name`. perhaps we should also qualify entrypoint (e.g. `tvm_unpacked_entrypoint_name`).

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -535,6 +551,8 @@ class AOTExecutorCodegen : public ExprVisitor {
   TargetsMap targets_;
   /*! \brief target host */
   Target target_host_;
+  /*! \brief untyped operators flag */

Review comment:
       could you explain this a bit more and specify polarity (e.g. true when abc thing should happen; false when other thing should happen)?

##########
File path: src/target/target_kind.cc
##########
@@ -219,6 +219,7 @@ TVM_REGISTER_TARGET_KIND("llvm", kDLCPU)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<String>("runtime")
     .add_attr_option<Bool>("link-params", Bool(false))
+    .add_attr_option<Bool>("unpacked-api", Bool(false))

Review comment:
       here i think we should not specify a default--this is why we see errors like https://ci.tlcpack.ai/blue/organizations/jenkins/tvm/detail/PR-8155/2/pipeline. These are identical to what we'll see when we migrate `unpacked-api` to a separate configuration space.
   
   ```
   Traceback (most recent call last):
     File "/usr/local/lib/python3.6/dist-packages/sphinx_gallery/gen_rst.py", line 480, in _memory_usage
       out = func()
     File "/usr/local/lib/python3.6/dist-packages/sphinx_gallery/gen_rst.py", line 465, in __call__
       exec(self.code, self.globals)
     File "/workspace/tutorials/get_started/auto_tuning_with_python.py", line 408, in <module>
       with autotvm.apply_history_best(tuning_option["tuning_records"]):
     File "/workspace/docs/../python/tvm/autotvm/task/dispatcher.py", line 201, in __init__
       self.load(records)
     File "/workspace/docs/../python/tvm/autotvm/task/dispatcher.py", line 229, in load
       for inp, res in records:
     File "/workspace/docs/../python/tvm/autotvm/record.py", line 212, in load_from_file
       ret = decode(row)
     File "/workspace/docs/../python/tvm/autotvm/record.py", line 157, in decode
       tgt = Target(str(tgt))
     File "/workspace/docs/../python/tvm/target/target.py", line 104, in __init__
       self.__init_handle_by_constructor__(_ffi_api.Target, target)
     File "tvm/_ffi/_cython/./object.pxi", line 126, in tvm._ffi._cy3.core.ObjectBase.__init_handle_by_constructor__
     File "tvm/_ffi/_cython/./packed_func.pxi", line 279, in tvm._ffi._cy3.core.ConstructorCall
     File "tvm/_ffi/_cython/./packed_func.pxi", line 257, in tvm._ffi._cy3.core.FuncCall
     File "tvm/_ffi/_cython/./packed_func.pxi", line 246, in tvm._ffi._cy3.core.FuncCall3
     File "tvm/_ffi/_cython/./base.pxi", line 163, in tvm._ffi._cy3.core.CALL
   ValueError: Traceback (most recent call last):
     5: TVMFuncCall
           at /workspace/src/runtime/c_runtime_api.cc:474
     4: tvm::runtime::PackedFunc::CallPacked(tvm::runtime::TVMArgs, tvm::runtime::TVMRetValue*) const
           at /workspace/include/tvm/runtime/packed_func.h:1150
     3: std::function<void (tvm::runtime::TVMArgs, tvm::runtime::TVMRetValue*)>::operator()(tvm::runtime::TVMArgs, tvm::runtime::TVMRetValue*) const
           at /usr/include/c++/5/functional:2267
     2: std::_Function_handler<void (tvm::runtime::TVMArgs, tvm::runtime::TVMRetValue*), void (*)(tvm::runtime::TVMArgs, tvm::runtime::TVMRetValue*)>::_M_invoke(std::_Any_data const&, tvm::runtime::TVMArgs&&, tvm::runtime::TVMRetValue*&&)
           at /usr/include/c++/5/functional:1871
     1: tvm::TargetInternal::ConstructorDispatcher(tvm::runtime::TVMArgs, tvm::runtime::TVMRetValue*)
           at /workspace/src/target/target.cc:496
     0: tvm::Target::Target(tvm::runtime::String const&)
           at /workspace/src/target/target.cc:391
     File "/workspace/src/target/target.cc", line 391
   
   ValueError: Error when parsing target["unpacked-api"]: Cannot recognize 'unpacked-api'. Candidates are: link-params, system-lib, model, mfloat-abi, mcpu, tag, mattr, runtime, device, keys, libs, host, mtriple. Target creation from string failed: llvm -keys=cpu -link-params=0 -unpacked-api=0
   ```




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --typed-operators

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



##########
File path: src/tir/transforms/make_unpacked_api.cc
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 make_unpacked_api.cc Lower PrimFunc to a standard C function API.
+ */
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/analysis.h>
+#include <tvm/tir/buffer.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/expr.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "arg_binder.h"
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+PrimFunc MakeUnpackedAPI(PrimFunc&& func) {
+  auto global_symbol = func->GetAttr<String>(tvm::attr::kGlobalSymbol);
+  ICHECK(global_symbol) << "MakeUnpackedAPI: Expect PrimFunc to have the global_symbol attribute";
+
+  auto target = func->GetAttr<Target>(tvm::attr::kTarget);
+  ICHECK(target.defined()) << "MakeUnpackedAPI: Require the target attribute";
+
+  auto* func_ptr = func.CopyOnWrite();
+
+  // Setup device context
+  int target_device_type = target.value()->kind->device_type;
+  Integer device_type(target_device_type);
+  Integer device_id(0);
+  PrimExpr node = StringImm("default");
+  const Stmt nop = Evaluate(0);
+  std::vector<Stmt> device_init;
+
+  // Create arg to buffer binder
+  std::unordered_map<const VarNode*, PrimExpr> vmap;
+  ArgBinder binder(&vmap);
+
+  // Collect variables and buffers to map between
+  Array<Var> args;
+  std::vector<std::pair<Var, Var>> var_def;
+  std::vector<std::pair<Var, Buffer>> buffer_def;
+
+  for (int i = 0; i < static_cast<int>(func_ptr->params.size()); ++i) {
+    Var param = func_ptr->params[i];

Review comment:
       `func_ptr->params` is an `tvm::runtime::Array<tvm::tir::Var>` so the value should always exist and always be a valid `Var`? 




-- 
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 #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       hrm couple thoughts:
   - `tvm_module_main` is supposed to be the default name of a function contained within a runtime.Module. It tends to be used with autotuning--the lone function being tuned is named `tvm_module_main`. I believe it's also used when Relay isn't used to build a model-level thing.
   - It's been called the "entrypoint" before, so it does seem like it would match
   - At minimum we'd need to ensure it becomes eventually prefixed to the module. I think that's being done in https://github.com/apache/tvm/pull/8014.
   - However, there is a thought from [Model-based Module Runtime Interface](https://discuss.tvm.apache.org/t/discuss-module-based-model-runtime-interface/5025) that the entrypoint should be the name of the model (as passed to tvm.relay.build). I don't know that anyone's made good on that yet, but that would be the most future-proof thing, I think (or at least, it would be most aligned with the forum).




-- 
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 #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: src/tir/transforms/make_unpacked_api.cc
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 make_unpacked_api.cc Lower PrimFunc to a standard C function API.
+ */
+#include <tvm/runtime/container.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/target.h>
+#include <tvm/tir/analysis.h>
+#include <tvm/tir/buffer.h>
+#include <tvm/tir/builtin.h>
+#include <tvm/tir/expr.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "arg_binder.h"
+#include "ir_utils.h"
+
+namespace tvm {
+namespace tir {
+
+PrimFunc MakeUnpackedAPI(PrimFunc&& func) {
+  auto global_symbol = func->GetAttr<String>(tvm::attr::kGlobalSymbol);
+  ICHECK(global_symbol) << "MakeUnpackedAPI: Expect PrimFunc to have the global_symbol attribute";
+
+  auto target = func->GetAttr<Target>(tvm::attr::kTarget);
+  ICHECK(target.defined()) << "MakeUnpackedAPI: Require the target attribute";
+
+  auto* func_ptr = func.CopyOnWrite();
+
+  // Setup device context
+  int target_device_type = target.value()->kind->device_type;
+  Integer device_type(target_device_type);
+  Integer device_id(0);
+  PrimExpr node = StringImm("default");
+  const Stmt nop = Evaluate(0);
+  std::vector<Stmt> device_init;
+
+  // Create arg to buffer binder
+  std::unordered_map<const VarNode*, PrimExpr> vmap;
+  ArgBinder binder(&vmap);
+
+  // Collect variables and buffers to map between
+  Array<Var> args;
+  std::vector<std::pair<Var, Var>> var_def;
+  std::vector<std::pair<Var, Buffer>> buffer_def;
+
+  for (int i = 0; i < static_cast<int>(func_ptr->params.size()); ++i) {
+    Var param = func_ptr->params[i];

Review comment:
       oh my apologies, you're right.




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

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



[GitHub] [tvm] areusch commented on pull request #8023: [AOT] Initial implementation of --unpacked-api

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


   thanks @Mousius, the PR is now 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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --no-typed-operators

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -605,7 +625,8 @@ class AOTExecutorCodegen : public ExprVisitor {
     // Find the return sid
     return_sid_ = AotReturnSidVisitor(storage_device_map_).FindReturnSid(func);
     for (unsigned int output_index = 0; output_index < return_sid_.size(); output_index++) {
-      main_signature_.push_back(tir::Var(MakeString("output_", output_index), DataType::Handle()));
+      auto output_var = tir::Var("output", DataType::Handle());
+      main_signature_.push_back(output_var);

Review comment:
       See above, though splitting it out into another variable was for something else I was doing so I'll fix that.




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --no-typed-operators

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -137,10 +137,17 @@ class AOTExecutorCodegen : public ExprVisitor {
       // Pack the sid inside the TVMValue
       auto sid_array = te::Var(MakeString("sid_", sid, "_value"), DataType::Handle());
       auto sid_value = sids_table_[sid];
-      tvm::PrimExpr set_tensor =
-          tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
-                         {sid_array, 0, tir::builtin::kArrData, sid_value});
-      stmts_.push_back(tir::LetStmt(sid_array, StackAlloca("array", 1), tir::Evaluate(set_tensor)));
+
+      if (target_host_->GetAttr<Bool>("typed-operators").value_or(Bool(true))) {

Review comment:
       Will do :+1:




-- 
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 #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       yeah there is some overlap between Module-based Model Runtime and the firmware-facing interface. the design priority for the firmware-facing interface is that it makes sense as an embedded-facing thing. however, i'm trying to keep it to be as similar to Module-based Model Runtime interface within those constraints.
   
   given this generated function is currently somewhat internal, i think we can use `tvm_module_main` for now and focus our efforts on the wrapping API code. should there be a case where multiple models are placed in the same module, we'll likely have to revisit that later on. but it's not clear we will support this, nor is it clear we would support it by allowing a single module to mix models. (there are, however, some compelling use cases, such as running two unrelated models in parallel).




-- 
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] Mousius commented on pull request #8023: [AOT] Initial implementation of --typed-operators

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


   @giuseros everythings back to green, could you take another look? :smile_cat: 


-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       Regards the Model-based Module Runtime Interface, this is also being approached as part of name mangling in the RFC?
   
   From the perspective of this PR, it makes more sense to use `tvm_module_main` and then allow the name to be mangled for use in the model-based runtime interface as part of #8014? Dependent on ordering of the merges ofc :smile_cat: 




-- 
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] Mousius commented on a change in pull request #8023: [AOT] Initial implementation of --unpacked-api

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



##########
File path: include/tvm/runtime/module.h
##########
@@ -232,6 +232,8 @@ constexpr const char* tvm_param_prefix = "__tvm_param__";
 constexpr const char* tvm_lookup_linked_param = "_lookup_linked_param";
 /*! \brief The main AOT executor function */
 constexpr const char* tvm_run_func_prefix = "tvm__run_func";
+/*! \brief The entrypoint function to the generated network */

Review comment:
       Sorry, what I meant was that the code to name mangle and allow multiple models is landing in #8014; that means we should use `tvm_module_main` here for now and it'll be replaced shortly by one with the model name included - thus providing the outcome we're looking for?
   
   Also, I believe we'll see multiple models in a single embedded application with increasing frequency and we can't leave out this use case.




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