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/03/26 00:44:53 UTC

[GitHub] [tvm] jroesch opened a new pull request #7746: Add support for using the VM across the RPC boundary.

jroesch opened a new pull request #7746:
URL: https://github.com/apache/tvm/pull/7746


   Refactor some of the internals of VM in order to enable use across the RPC boundary. I need to remove the use of the `lib` field to instead use the imports completely still.
   
   cc @zhiics @adelbertc @tmoreau89 @tkonolige @jwfromm @michalpiszczek 


-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/library_module.cc
##########
@@ -106,16 +106,16 @@ Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {
   if (f == nullptr) {
     std::string loaders = "";
     for (auto name : Registry::ListNames()) {
-      if (name.rfind(loadkey, 0) == 0) {
+      if (name.find(loadkey, 0) == 0) {
         if (loaders.size() > 0) {
           loaders += ", ";
         }
         loaders += name.substr(loadkey.size());
       }
     }
-    ICHECK(f != nullptr) << "Binary was created using " << type_key
-                         << " but a loader of that name is not registered. Available loaders are "
-                         << loaders << ". Perhaps you need to recompile with this runtime enabled.";
+    LOG(FATAL) << "Binary was created using " << type_key

Review comment:
       Because `if (cond) { }; CHECK(!cond) << msg;` is equivalent to `if (cond) else { CHECK(false) << msg }` which also is equivalent to `if (cond) { } LOG(FATAL) << msg`




-- 
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 #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: python/tvm/runtime/vm.py
##########
@@ -299,12 +300,16 @@ class VirtualMachine(object):
     POOLED_ALLOCATOR = 2
 
     def __init__(self, exe, device, memory_cfg=None):

Review comment:
       can you do the last part here?

##########
File path: src/runtime/library_module.cc
##########
@@ -106,16 +106,16 @@ Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {
   if (f == nullptr) {
     std::string loaders = "";
     for (auto name : Registry::ListNames()) {
-      if (name.rfind(loadkey, 0) == 0) {
+      if (name.find(loadkey, 0) == 0) {
         if (loaders.size() > 0) {
           loaders += ", ";
         }
         loaders += name.substr(loadkey.size());
       }
     }
-    ICHECK(f != nullptr) << "Binary was created using " << type_key
-                         << " but a loader of that name is not registered. Available loaders are "
-                         << loaders << ". Perhaps you need to recompile with this runtime enabled.";
+    LOG(FATAL) << "Binary was created using " << type_key

Review comment:
       why LOG(FATAL)?

##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -125,9 +133,17 @@ class Executable : public ModuleNode {
    * \brief Get the `lib` module in an executable. Users have the flexibility to call
    * `export_library` from the frontend to save the library to disk.
    *
-   * \return The runtime module that contains the hardwre dependent code.
+   * \return The runtime module that contains the hardware dependent code.
    */
-  runtime::Module GetLib() const { return lib; }
+  runtime::Module GetLib() const { return this->imports_[0]; }
+
+  void SetLib(const runtime::Module& lib) {

Review comment:
       isn't Import a public function, is what I'm saying? SetLib is effectively an internal function used from import, correct?

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -1155,18 +1155,20 @@ void VMCompiler::Codegen() {
 
   auto compile_engine = CompileEngine::Global();
   auto ext_mods = compile_engine->LowerExternalFunctions();
+  runtime::Module lib;
   if (funcs.size() > 0) {
     Map<String, IRModule> build_funcs;
     for (const auto& i : funcs) {
       build_funcs.Set(i.first, i.second);
     }
-    exec_->lib = tvm::build(build_funcs, target_host_);
+    lib = tvm::build(build_funcs, target_host_);
   } else {
     // There is no function handled by TVM. We create a virtual main module
     // to make sure a DSO module will be also available.
-    exec_->lib = codegen::CSourceModuleCreate(";", "", Array<String>{});
+    lib = codegen::CSourceModuleCreate(";", "", Array<String>{});
   }
-  exec_->lib = codegen::CreateMetadataModule(params_, exec_->lib, ext_mods, target_host_);
+  lib = codegen::CreateMetadataModule(params_, lib, ext_mods, target_host_);
+  exec_->SetLib(lib);

Review comment:
       ?




-- 
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 #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -63,6 +63,14 @@ class Executable : public ModuleNode {
    */
   PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) final;
 
+  /*!
+   * \brief Save the entire executable to a binary stream.
+   * \param stream The binary stream to save to.
+   */
+  void SaveToBinary(dmlc::Stream* stream) final;
+
+  void SaveToFile(const std::string& path, const std::string& format) final;

Review comment:
       docstring

##########
File path: python/tvm/runtime/module.py
##########
@@ -269,24 +269,31 @@ def _collect_dso_modules(self):
         return self._collect_from_import_tree(is_dso_exportable)
 
     def export_library(self, file_name, fcompile=None, addons=None, workspace_dir=None, **kwargs):
-        """Export the module and its imported device code one library.
+        """
+        Export the module and all imported modules into a single device library.
 
-        This function only works on host llvm modules.
-        It will pack all the imported modules
+        This function only works on hos LLVM modules, other runtime::Module
+        subclasses will work with this API but they must support implement
+        the save and load mechanisms of modules completely including saving
+        from streams and files. This will pack your non-shared library module
+        into a single shared library which can later be loaded by TVM.
 
         Parameters
         ----------
         file_name : str
             The name of the shared library.
 
         fcompile : function(target, file_list, kwargs), optional
-            Compilation function to use create dynamic library.
+            The compilation function to use create the final library object during
+            export. For example this is used to link together all produced artifacts

Review comment:
       For example, when fcompile=_cc.create_shared, or when it is not supplied but module is "llvm," 

##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -125,9 +133,17 @@ class Executable : public ModuleNode {
    * \brief Get the `lib` module in an executable. Users have the flexibility to call
    * `export_library` from the frontend to save the library to disk.
    *
-   * \return The runtime module that contains the hardwre dependent code.
+   * \return The runtime module that contains the hardware dependent code.
    */
-  runtime::Module GetLib() const { return lib; }
+  runtime::Module GetLib() const { return this->imports_[0]; }
+
+  void SetLib(const runtime::Module& lib) {

Review comment:
       why do you do this rather than just calling Import() from a factory function?

##########
File path: python/tvm/runtime/module.py
##########
@@ -269,24 +269,31 @@ def _collect_dso_modules(self):
         return self._collect_from_import_tree(is_dso_exportable)
 
     def export_library(self, file_name, fcompile=None, addons=None, workspace_dir=None, **kwargs):

Review comment:
       want to add type annotations?

##########
File path: src/runtime/vm/executable.cc
##########
@@ -74,6 +76,12 @@ PackedFunc Executable::GetFunction(const std::string& name, const ObjectPtr<Obje
       int index = args[1];
       *rv = this->GetFunctionParameterName(func_name, index);
     });
+  } else if (name == "vm_load_executable") {

Review comment:
       why is this a member function of Executable rather than a factory function for VM?

##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -125,9 +133,17 @@ class Executable : public ModuleNode {
    * \brief Get the `lib` module in an executable. Users have the flexibility to call
    * `export_library` from the frontend to save the library to disk.
    *
-   * \return The runtime module that contains the hardwre dependent code.
+   * \return The runtime module that contains the hardware dependent code.
    */
-  runtime::Module GetLib() const { return lib; }
+  runtime::Module GetLib() const { return this->imports_[0]; }

Review comment:
       explain why it is always imports_[0]

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -1155,18 +1155,20 @@ void VMCompiler::Codegen() {
 
   auto compile_engine = CompileEngine::Global();
   auto ext_mods = compile_engine->LowerExternalFunctions();
+  runtime::Module lib;
   if (funcs.size() > 0) {
     Map<String, IRModule> build_funcs;
     for (const auto& i : funcs) {
       build_funcs.Set(i.first, i.second);
     }
-    exec_->lib = tvm::build(build_funcs, target_host_);
+    lib = tvm::build(build_funcs, target_host_);
   } else {
     // There is no function handled by TVM. We create a virtual main module
     // to make sure a DSO module will be also available.
-    exec_->lib = codegen::CSourceModuleCreate(";", "", Array<String>{});
+    lib = codegen::CSourceModuleCreate(";", "", Array<String>{});
   }
-  exec_->lib = codegen::CreateMetadataModule(params_, exec_->lib, ext_mods, target_host_);
+  lib = codegen::CreateMetadataModule(params_, lib, ext_mods, target_host_);
+  exec_->SetLib(lib);

Review comment:
       here I think you can just `exec_->Import(lib)`?

##########
File path: python/tvm/runtime/vm.py
##########
@@ -299,12 +300,16 @@ class VirtualMachine(object):
     POOLED_ALLOCATOR = 2
 
     def __init__(self, exe, device, memory_cfg=None):
-        if not isinstance(exe, Executable):
+        if not isinstance(exe, Executable) and not isinstance(exe, Module):

Review comment:
       you don't need the first condition here

##########
File path: src/runtime/vm/executable.cc
##########
@@ -476,8 +484,19 @@ void LoadHeader(dmlc::Stream* strm) {
 }
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
+  std::cout << "code: " << code.size() << std::endl;

Review comment:
       rm

##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -63,6 +63,14 @@ class Executable : public ModuleNode {
    */
   PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) final;
 
+  /*!
+   * \brief Save the entire executable to a binary stream.
+   * \param stream The binary stream to save to.
+   */
+  void SaveToBinary(dmlc::Stream* stream) final;
+
+  void SaveToFile(const std::string& path, const std::string& format) final;

Review comment:
       docstring

##########
File path: src/runtime/library_module.cc
##########
@@ -99,6 +99,28 @@ void InitContextFunctions(std::function<void*(const char*)> fgetsymbol) {
 #undef TVM_INIT_CONTEXT_FUNC
 }
 
+Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {

Review comment:
       add a docstring and explain when this is used

##########
File path: python/tvm/runtime/vm.py
##########
@@ -299,12 +300,16 @@ class VirtualMachine(object):
     POOLED_ALLOCATOR = 2
 
     def __init__(self, exe, device, memory_cfg=None):

Review comment:
       add a docstring, maybe type annotations, and explain in what case exe would be a Module

##########
File path: src/runtime/vm/executable.cc
##########
@@ -476,8 +484,19 @@ void LoadHeader(dmlc::Stream* strm) {
 }
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
+  std::cout << "code: " << code.size() << std::endl;
   auto exec = make_object<Executable>();
-  exec->lib = lib;
+
+  // Support null-initialization of lib, to enable initialization during
+  // deserialization before we have we have deserialized the imports.
+  if (lib.defined()) {

Review comment:
       perhaps it's worth either creating a const bool is_initialized() { ICHECK_LE(this->imports_.size(), 1); return this->imports_.size() == 1; } and adding it everywhere we access lib, or make the change I suggested above in GetLib and audit all uses of lib to ensure we GetLib always.

##########
File path: tests/python/relay/test_vm.py
##########
@@ -799,5 +802,30 @@ def test_constant_shape_with_external_codegen():
     assert "shape_func" in opt_mod.astext(False)
 
 
+def test_vm_rpc():
+    target = "llvm"
+    target_host = "llvm"
+
+    x = relay.var("x", shape=(10, 1))
+    f = relay.Function([x], x + x)
+    mod = IRModule.from_expr(f)
+    vm_exec = vm.compile(mod, target=target, target_host=target_host)
+
+    temp = utils.tempdir()

Review comment:
       can you just add a couple comments explaining what it is you're doing and what you're testing?

##########
File path: python/tvm/runtime/module.py
##########
@@ -269,24 +269,31 @@ def _collect_dso_modules(self):
         return self._collect_from_import_tree(is_dso_exportable)
 
     def export_library(self, file_name, fcompile=None, addons=None, workspace_dir=None, **kwargs):
-        """Export the module and its imported device code one library.
+        """
+        Export the module and all imported modules into a single device library.
 
-        This function only works on host llvm modules.
-        It will pack all the imported modules
+        This function only works on hos LLVM modules, other runtime::Module

Review comment:
       nit: host

##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -125,9 +133,17 @@ class Executable : public ModuleNode {
    * \brief Get the `lib` module in an executable. Users have the flexibility to call
    * `export_library` from the frontend to save the library to disk.
    *
-   * \return The runtime module that contains the hardwre dependent code.
+   * \return The runtime module that contains the hardware dependent code.
    */
-  runtime::Module GetLib() const { return lib; }
+  runtime::Module GetLib() const { return this->imports_[0]; }

Review comment:
       also add an ICHECK_EQ(this.imports_.size(), 1) and an explanatory error describing the programming error that causes 0 imports

##########
File path: src/runtime/vm/executable.cc
##########
@@ -476,8 +484,19 @@ void LoadHeader(dmlc::Stream* strm) {
 }
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
+  std::cout << "code: " << code.size() << std::endl;
   auto exec = make_object<Executable>();
-  exec->lib = lib;
+
+  // Support null-initialization of lib, to enable initialization during
+  // deserialization before we have we have deserialized the imports.
+  if (lib.defined()) {
+    ICHECK_EQ(exec->imports_.size(), 0)

Review comment:
       maybe this should move to GetLib. Since you just made exec, it should not have imports?




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/vm/executable.cc
##########
@@ -476,8 +484,19 @@ void LoadHeader(dmlc::Stream* strm) {
 }
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
+  std::cout << "code: " << code.size() << std::endl;

Review comment:
       got 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] jroesch commented on pull request #7746: Add support for using the VM across the RPC boundary.

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


   The last commit resolves all the outstanding comments, we will wait to be on 3.7 to use type annotations and I added doc comments and clarifications. 


-- 
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] jroesch merged pull request #7746: Add support for using the VM across the RPC boundary.

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


   


-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: python/tvm/runtime/vm.py
##########
@@ -299,12 +300,16 @@ class VirtualMachine(object):
     POOLED_ALLOCATOR = 2
 
     def __init__(self, exe, device, memory_cfg=None):
-        if not isinstance(exe, Executable):
+        if not isinstance(exe, Executable) and not isinstance(exe, Module):

Review comment:
       I do because the Python class doesn't inherit from Module unfortunately due to the way the wrappers are written. 




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/library_module.cc
##########
@@ -99,6 +99,29 @@ void InitContextFunctions(std::function<void*(const char*)> fgetsymbol) {
 #undef TVM_INIT_CONTEXT_FUNC
 }
 
+Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {
+  std::string loadkey = "runtime.module.loadbinary_";
+  std::string fkey = loadkey + type_key;
+  const PackedFunc* f = Registry::Get(fkey);
+  if (f == nullptr) {
+    std::string loaders = "";
+    for (auto name : Registry::ListNames()) {
+      if (name.rfind(loadkey, 0) == 0) {

Review comment:
       This is existing code I just pulled into a function because too much inline code offends my taste. 




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/vm/executable.cc
##########
@@ -476,8 +484,19 @@ void LoadHeader(dmlc::Stream* strm) {
 }
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
+  std::cout << "code: " << code.size() << std::endl;
   auto exec = make_object<Executable>();
-  exec->lib = lib;
+
+  // Support null-initialization of lib, to enable initialization during
+  // deserialization before we have we have deserialized the imports.
+  if (lib.defined()) {

Review comment:
       The problem is in this case we actually need to check the argument, arguably it should be like Optional<runtime::Module> but since Module is nullable you have to check for inner nullability either way. I don't think that would clean up this use site, I made the change in GetLib and will make that and SetLib preferred interface at least until we figure out aritfact situation. 




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: python/tvm/runtime/module.py
##########
@@ -269,24 +269,31 @@ def _collect_dso_modules(self):
         return self._collect_from_import_tree(is_dso_exportable)
 
     def export_library(self, file_name, fcompile=None, addons=None, workspace_dir=None, **kwargs):

Review comment:
       Are we doing that now? I wasn't sure if we are in annotation world yet




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/vm/executable.cc
##########
@@ -74,6 +76,12 @@ PackedFunc Executable::GetFunction(const std::string& name, const ObjectPtr<Obje
       int index = args[1];
       *rv = this->GetFunctionParameterName(func_name, index);
     });
+  } else if (name == "vm_load_executable") {

Review comment:
       It is a factory function in C++, I am using the Executable to act directly as a factory module instead of needing to introduce a 3rd concept like the GraphRuntimeFactory does. 




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

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



[GitHub] [tvm] zhiics commented on pull request #7746: Add support for using the VM across the RPC boundary.

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


   please fix the linting error again.


-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/vm/executable.cc
##########
@@ -477,7 +485,17 @@ void LoadHeader(dmlc::Stream* strm) {
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
   auto exec = make_object<Executable>();
-  exec->lib = lib;
+
+  // Support null-initialization of lib, to enable initialization during
+  // deserialization before we have we have deserialized the imports.
+  if (lib.defined()) {
+    ICHECK_EQ(exec->imports_.size(), 0)
+        << "A VMExecutable should never have more than one import inside an the executable, \n"
+        << "the first import should *always* be the library containing"
+        << "the platform specific kernel code";
+    exec->Import(lib);

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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: include/tvm/runtime/vm/executable.h
##########
@@ -125,9 +133,17 @@ class Executable : public ModuleNode {
    * \brief Get the `lib` module in an executable. Users have the flexibility to call
    * `export_library` from the frontend to save the library to disk.
    *
-   * \return The runtime module that contains the hardwre dependent code.
+   * \return The runtime module that contains the hardware dependent code.
    */
-  runtime::Module GetLib() const { return lib; }
+  runtime::Module GetLib() const { return this->imports_[0]; }
+
+  void SetLib(const runtime::Module& lib) {

Review comment:
       I wrote this comment later in the code but we need to support partial initialization for the serialization case, if we rework how Module serialization works we could remove this but now I want to ensure we initialize it correctly and also provide an interface so we don't have to rewrite all the code if we iterate on this interface over the next few months. 




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/vm/executable.cc
##########
@@ -476,8 +484,19 @@ void LoadHeader(dmlc::Stream* strm) {
 }
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
+  std::cout << "code: " << code.size() << std::endl;
   auto exec = make_object<Executable>();
-  exec->lib = lib;
+
+  // Support null-initialization of lib, to enable initialization during
+  // deserialization before we have we have deserialized the imports.
+  if (lib.defined()) {
+    ICHECK_EQ(exec->imports_.size(), 0)

Review comment:
       It shouldn't but I'm a defensive programmer and paranoid about people wrongfactoring the code later in a way that breaks an invariant. 




-- 
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] jroesch commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/library_module.cc
##########
@@ -99,6 +99,28 @@ void InitContextFunctions(std::function<void*(const char*)> fgetsymbol) {
 #undef TVM_INIT_CONTEXT_FUNC
 }
 
+Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {

Review comment:
       The docstring is in the header file




-- 
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] tkonolige commented on a change in pull request #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/library_module.cc
##########
@@ -99,6 +99,29 @@ void InitContextFunctions(std::function<void*(const char*)> fgetsymbol) {
 #undef TVM_INIT_CONTEXT_FUNC
 }
 
+Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {
+  std::string loadkey = "runtime.module.loadbinary_";
+  std::string fkey = loadkey + type_key;
+  const PackedFunc* f = Registry::Get(fkey);
+  if (f == nullptr) {
+    std::string loaders = "";
+    for (auto name : Registry::ListNames()) {
+      if (name.rfind(loadkey, 0) == 0) {
+        if (loaders.size() > 0) {
+          loaders += ", ";
+        }
+        loaders += name.substr(loadkey.size());
+      }
+    }
+    ICHECK(f != nullptr)

Review comment:
       `ICHECK_NOTNULL(f)` or `LOG(FATAL)`

##########
File path: src/runtime/library_module.cc
##########
@@ -99,6 +99,28 @@ void InitContextFunctions(std::function<void*(const char*)> fgetsymbol) {
 #undef TVM_INIT_CONTEXT_FUNC
 }
 
+Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {
+  std::string loadkey = "runtime.module.loadbinary_";
+  std::string fkey = loadkey + type_key;
+  const PackedFunc* f = Registry::Get(fkey);
+  if (f == nullptr) {
+    std::string loaders = "";
+    for (auto name : Registry::ListNames()) {
+      if (name.rfind(loadkey, 0) == 0) {
+        if (loaders.size() > 0) {
+          loaders += ", ";
+        }
+        loaders += name.substr(loadkey.size());
+      }
+    }
+    ICHECK(f != nullptr) << "Binary was created using " << type_key

Review comment:
       `ICHECK_NOTNULL` or `LOG(FATAL)`

##########
File path: src/runtime/library_module.cc
##########
@@ -99,6 +99,29 @@ void InitContextFunctions(std::function<void*(const char*)> fgetsymbol) {
 #undef TVM_INIT_CONTEXT_FUNC
 }
 
+Module LoadModuleFromBinary(const std::string& type_key, dmlc::Stream* stream) {
+  std::string loadkey = "runtime.module.loadbinary_";
+  std::string fkey = loadkey + type_key;
+  const PackedFunc* f = Registry::Get(fkey);
+  if (f == nullptr) {
+    std::string loaders = "";
+    for (auto name : Registry::ListNames()) {
+      if (name.rfind(loadkey, 0) == 0) {

Review comment:
       ```suggestion
         if (name.find(loadkey, 0) == 0) {
   ```
   
   Assuming you want the string to start with "runtime.module.loadbinary_".




-- 
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 #7746: Add support for using the VM across the RPC boundary.

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



##########
File path: src/runtime/vm/executable.cc
##########
@@ -477,7 +485,17 @@ void LoadHeader(dmlc::Stream* strm) {
 
 runtime::Module Executable::Load(const std::string& code, const runtime::Module lib) {
   auto exec = make_object<Executable>();
-  exec->lib = lib;
+
+  // Support null-initialization of lib, to enable initialization during
+  // deserialization before we have we have deserialized the imports.
+  if (lib.defined()) {
+    ICHECK_EQ(exec->imports_.size(), 0)
+        << "A VMExecutable should never have more than one import inside an the executable, \n"
+        << "the first import should *always* be the library containing"
+        << "the platform specific kernel code";
+    exec->Import(lib);

Review comment:
       use `exec->SetLib(lib)`?




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