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 2022/07/29 09:07:18 UTC

[GitHub] [tvm] juda opened a new pull request, #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

juda opened a new pull request, #12232:
URL: https://github.com/apache/tvm/pull/12232

   This PR solves two issues:
   
   1. The compatibility of libstdc++ CXX11 ABI. 
       Currently, the official PyTorch distribution uses old symbols from libstdc++, which conflicts with the symbols used by TVM. The issue was discussed [here](https://discuss.tvm.apache.org/t/can-someone-please-give-me-the-steps-to-use-pt-tvmdsoop/12525) before. 
       We address this issue by compiling the code snippets involving TVM and Torch separately, with their right libstdc++ CXX ABI. The TVM-related codes ([RuntimeModuleWrapperTVM.cc](https://github.com/juda/tvm/pull/4/files#diff-3cb0a7daf9d8032f468f8fda43a9f0a2a94f7c7cd8126c9d6f09b4a805c3c2d0)) are built under the new CXX11 ABI, while the Torch-related codes ([RuntimeModuleWrapperTorch.cc](https://github.com/juda/tvm/pull/4/files#diff-7a2704d021b5f88143cedfbc3bb2b24d289c3616112c0c1e1dc61c29f1881e19)) are built under the same CXX11 ABI as the installed PyTorch, and linked together by a pure C header ([runtime_bridge.h](https://github.com/juda/tvm/pull/4/files#diff-6b8efa6b4cc714cb50d042cccc4c268cc54cc827d832eda1395885a372cd9b12)).
   
   2. The lack of the support of boolean tensor.
       Currently, If we tried to use `optimze_torch` with an input of boolean tensor, it will fail because it’s not supported by DLPack (https://github.com/dmlc/dlpack/issues/75). We might want to work around it since some models use boolean tensor.
       We address this issue by extending the DLTensor with an extra `is_bool` field, guiding us to convert NDArray and DLTensor with the correct type. If the DLTensor is not boolean, the behavior of data transformation is the same as the previous codes.


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] yelite commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
yelite commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r933381946


##########
cmake/modules/contrib/PT_TVMDSOOP.cmake:
##########
@@ -21,38 +21,55 @@ if(NOT USE_PT_TVMDSOOP STREQUAL "OFF")
   execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch; print(torch.__path__[0].strip())"
     OUTPUT_VARIABLE PT_PATH
     RESULT_VARIABLE PT_STATUS)
-  if (NOT ${PT_STATUS} EQUAL 0)
+
+  if(NOT ${PT_STATUS} EQUAL 0)
     message(FATAL_ERROR "Fail to get pytorch path")
   endif()
 
   string(REGEX REPLACE "\n" "" PT_PATH "${PT_PATH}")
   message(STATUS "PyTorch path: ${PT_PATH}")
 
-  set(PT_COMPILE_FLAGS_STR "-I${PT_PATH}/include -D_GLIBCXX_USE_CXX11_ABI=0")
+  execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch;print(torch.compiled_with_cxx11_abi())"
+    OUTPUT_VARIABLE PT_CXX_FLAG
+    RESULT_VARIABLE PT_STATUS)
+
+  string(REGEX REPLACE "\n" "" PT_CXX_FLAG "${PT_CXX_FLAG}")
+  message(STATUS "Found TORCH_BUILT_WITH_CXX_ABI=${PT_CXX_FLAG} ")
+
+  if(${PT_CXX_FLAG} STREQUAL "False")
+    set(CXX_ABI_ENABLED 0)
+  else()
+    set(CXX_ABI_ENABLED 1)
+  endif()
+
+  set_property(
+    SOURCE
+    ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc
+    APPEND PROPERTY
+    COMPILE_OPTIONS
+    "-D_GLIBCXX_USE_CXX11_ABI=${CXX_ABI_ENABLED}"
+    "-I${PT_PATH}/include"
+  )
   set(PT_LINK_FLAGS_STR "-L${PT_PATH}/lib -l:libtorch.so -l:libtorch_python.so")
 
   if(NOT USE_CUDA STREQUAL "OFF")
     add_definitions(-DPT_TVMDSOOP_ENABLE_GPU)
   endif()
 
-
   string(REGEX REPLACE "\n" " " PT_FLAGS "${PT_COMPILE_FLAGS} ${PT_LINK_FLAGS}")
-  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND ${PT_COMPILE_FLAGS_STR})
+  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND)
   separate_arguments(PT_LINK_FLAGS UNIX_COMMAND ${PT_LINK_FLAGS_STR})
 
-
   set(LIBRARY_NAME pt_tvmdsoop)
-  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/**/*.cc)
+  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/*.cc)

Review Comment:
   Instead of skipping compilation of `tvm_class.cc`, can you move the new code to a *separate dynamic library*? This can be done by adding a new `add_library` command and other related commands to the cmake file. This means, in addition to `libpt_tvmdsoop`, the build will produce another library for the new PyTorch integration. Then in the Python code, it should load those two libraries in two try-catch blocks (log error and continue if `dlopen` 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934117229


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);

Review Comment:
   Done: saved the `runtime_module`



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934074843


##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;

Review Comment:
   Fixed



##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;
+
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+struct TVMContribTorchRuntimeModule;
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();

Review Comment:
   Fixed



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934081517


##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;
+
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+struct TVMContribTorchRuntimeModule;
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size);
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs);

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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934084833


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }
+
+  void forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+    tvm_contrib_torch_operator_module_forward(
+        this->runtime_module, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size());

Review Comment:
   No



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934087537


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};

Review Comment:
   I switch it to another syntax



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] yelite commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
yelite commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r933381946


##########
cmake/modules/contrib/PT_TVMDSOOP.cmake:
##########
@@ -21,38 +21,55 @@ if(NOT USE_PT_TVMDSOOP STREQUAL "OFF")
   execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch; print(torch.__path__[0].strip())"
     OUTPUT_VARIABLE PT_PATH
     RESULT_VARIABLE PT_STATUS)
-  if (NOT ${PT_STATUS} EQUAL 0)
+
+  if(NOT ${PT_STATUS} EQUAL 0)
     message(FATAL_ERROR "Fail to get pytorch path")
   endif()
 
   string(REGEX REPLACE "\n" "" PT_PATH "${PT_PATH}")
   message(STATUS "PyTorch path: ${PT_PATH}")
 
-  set(PT_COMPILE_FLAGS_STR "-I${PT_PATH}/include -D_GLIBCXX_USE_CXX11_ABI=0")
+  execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch;print(torch.compiled_with_cxx11_abi())"
+    OUTPUT_VARIABLE PT_CXX_FLAG
+    RESULT_VARIABLE PT_STATUS)
+
+  string(REGEX REPLACE "\n" "" PT_CXX_FLAG "${PT_CXX_FLAG}")
+  message(STATUS "Found TORCH_BUILT_WITH_CXX_ABI=${PT_CXX_FLAG} ")
+
+  if(${PT_CXX_FLAG} STREQUAL "False")
+    set(CXX_ABI_ENABLED 0)
+  else()
+    set(CXX_ABI_ENABLED 1)
+  endif()
+
+  set_property(
+    SOURCE
+    ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc
+    APPEND PROPERTY
+    COMPILE_OPTIONS
+    "-D_GLIBCXX_USE_CXX11_ABI=${CXX_ABI_ENABLED}"
+    "-I${PT_PATH}/include"
+  )
   set(PT_LINK_FLAGS_STR "-L${PT_PATH}/lib -l:libtorch.so -l:libtorch_python.so")
 
   if(NOT USE_CUDA STREQUAL "OFF")
     add_definitions(-DPT_TVMDSOOP_ENABLE_GPU)
   endif()
 
-
   string(REGEX REPLACE "\n" " " PT_FLAGS "${PT_COMPILE_FLAGS} ${PT_LINK_FLAGS}")
-  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND ${PT_COMPILE_FLAGS_STR})
+  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND)
   separate_arguments(PT_LINK_FLAGS UNIX_COMMAND ${PT_LINK_FLAGS_STR})
 
-
   set(LIBRARY_NAME pt_tvmdsoop)
-  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/**/*.cc)
+  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/*.cc)

Review Comment:
   Instead of skipping compilation of `tvm_class.cc`, can you move the new code to a *separate dynamic library*? This can be done by adding a new `add_library` command to the cmake file. This means, in addition to `libpt_tvmdsoop`, the build will produce another library for the new PyTorch integration. Then in the Python code, it should load those two libraries in two try-catch blocks (log error and continue if `dlopen` 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934107880


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  tvm::runtime::PackedFunc run = runtime_module.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);

Review Comment:
   The logic looks correct. 
   In PyTorch, an expression `(~a).bool()` will return `[False]` when `a = [True]` while it will return `[True]` when `a = [1]`. That is (one of) the different behavior for boolean and interger types.
   The `optimized_torch` is able to pass the tests, implying the logic is not found problematically.



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] masahi merged pull request #12232: [TVM PyTorch Integration] libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
masahi merged PR #12232:
URL: https://github.com/apache/tvm/pull/12232


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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934087603


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  tvm::runtime::PackedFunc run = runtime_module.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+
+  run();
+
+  int64_t output_length = get_num_outputs();
+
+  auto outputs_ptr = new DLPackTensorExt[output_length];
+  *outputs = outputs_ptr;
+
+  for (int k = 0; k < output_length; ++k) {
+    tvm::runtime::NDArray results = get_output(k);
+    auto is_bool = results.DataType().is_bool();
+    DLManagedTensor* tensor;
+    if (is_bool) {
+      auto tmp =
+          tvm::runtime::NDArray::Empty(results.Shape(), DLDataType{kDLInt, 8, 1}, device_info);
+      results.CopyTo(tmp);
+      tensor = tmp.ToDLPack();
+    } else {
+      tensor = results.ToDLPack();
+    }

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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934084691


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {

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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934092594


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  tvm::runtime::PackedFunc run = runtime_module.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+
+  run();
+
+  int64_t output_length = get_num_outputs();
+
+  auto outputs_ptr = new DLPackTensorExt[output_length];
+  *outputs = outputs_ptr;
+
+  for (int k = 0; k < output_length; ++k) {
+    tvm::runtime::NDArray results = get_output(k);
+    auto is_bool = results.DataType().is_bool();
+    DLManagedTensor* tensor;
+    if (is_bool) {
+      auto tmp =
+          tvm::runtime::NDArray::Empty(results.Shape(), DLDataType{kDLInt, 8, 1}, device_info);
+      results.CopyTo(tmp);
+      tensor = tmp.ToDLPack();
+    } else {
+      tensor = results.ToDLPack();
+    }
+    outputs_ptr[k] = {.dl_managed_tensor = tensor, .is_bool = is_bool};
+  }
+
+  return output_length;
+}
+
+char* tvm_contrib_torch_encode(TVMContribTorchRuntimeModule* runtime_module) {
+  auto std = tvm::contrib::serialize(runtime_module->mod);
+  auto* ret = new char[std.length() + 1];

Review Comment:
   Fixed



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r933779749


##########
cmake/modules/contrib/PT_TVMDSOOP.cmake:
##########
@@ -21,38 +21,55 @@ if(NOT USE_PT_TVMDSOOP STREQUAL "OFF")
   execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch; print(torch.__path__[0].strip())"
     OUTPUT_VARIABLE PT_PATH
     RESULT_VARIABLE PT_STATUS)
-  if (NOT ${PT_STATUS} EQUAL 0)
+
+  if(NOT ${PT_STATUS} EQUAL 0)
     message(FATAL_ERROR "Fail to get pytorch path")
   endif()
 
   string(REGEX REPLACE "\n" "" PT_PATH "${PT_PATH}")
   message(STATUS "PyTorch path: ${PT_PATH}")
 
-  set(PT_COMPILE_FLAGS_STR "-I${PT_PATH}/include -D_GLIBCXX_USE_CXX11_ABI=0")
+  execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch;print(torch.compiled_with_cxx11_abi())"
+    OUTPUT_VARIABLE PT_CXX_FLAG
+    RESULT_VARIABLE PT_STATUS)
+
+  string(REGEX REPLACE "\n" "" PT_CXX_FLAG "${PT_CXX_FLAG}")
+  message(STATUS "Found TORCH_BUILT_WITH_CXX_ABI=${PT_CXX_FLAG} ")
+
+  if(${PT_CXX_FLAG} STREQUAL "False")
+    set(CXX_ABI_ENABLED 0)
+  else()
+    set(CXX_ABI_ENABLED 1)
+  endif()
+
+  set_property(
+    SOURCE
+    ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc
+    APPEND PROPERTY
+    COMPILE_OPTIONS
+    "-D_GLIBCXX_USE_CXX11_ABI=${CXX_ABI_ENABLED}"
+    "-I${PT_PATH}/include"
+  )
   set(PT_LINK_FLAGS_STR "-L${PT_PATH}/lib -l:libtorch.so -l:libtorch_python.so")
 
   if(NOT USE_CUDA STREQUAL "OFF")
     add_definitions(-DPT_TVMDSOOP_ENABLE_GPU)
   endif()
 
-
   string(REGEX REPLACE "\n" " " PT_FLAGS "${PT_COMPILE_FLAGS} ${PT_LINK_FLAGS}")
-  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND ${PT_COMPILE_FLAGS_STR})
+  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND)
   separate_arguments(PT_LINK_FLAGS UNIX_COMMAND ${PT_LINK_FLAGS_STR})
 
-
   set(LIBRARY_NAME pt_tvmdsoop)
-  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/**/*.cc)
+  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/*.cc)

Review Comment:
   Done



##########
python/tvm/contrib/torch/pytorch_tvm.py:
##########
@@ -183,6 +184,11 @@ def load_tvm(self, export_dir):
 
     def build_pytorch_module(self, num_inputs, num_outputs, input_infos=None):
         """Build pytorch module containing TVM Graph Module"""
+        warnings.warn(
+            "We suggest users to use `optimized_torch` for tuning Torch modules instead",

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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934084309


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }
+
+  void forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+    tvm_contrib_torch_operator_module_forward(
+        this->runtime_module, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size());
+
+    for (int k = 0; k < input_length; ++k) {
+      tensors[k].dl_managed_tensor->deleter(tensors[k].dl_managed_tensor);
+    }
+  }
+
+  std::string Serialize() { return std::string(tvm_contrib_torch_encode(runtime_module)); }
+
+  explicit OperatorModuleWrapper(std::string state) {
+    runtime_module = tvm_contrib_torch_decode(state.c_str());
+  }
+
+ private:
+  TVMContribTorchRuntimeModule* runtime_module;
+};
+
+/**
+ * @brief A Torch's module which wraps TVM's GraphExecutorFactory Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class GraphExecutorFactoryWrapper : public torch::jit::CustomClassHolder {
+ public:
+  explicit GraphExecutorFactoryWrapper(TVMContribTorchRuntimeModule* executor_factory)
+      : executor_factory_(executor_factory) {}
+
+  GraphExecutorFactoryWrapper()
+      : GraphExecutorFactoryWrapper(tvm_contrib_torch_get_last_saved_runtime_module()) {}
+  std::string Serialize() { return tvm_contrib_torch_encode(executor_factory_); }
+
+  explicit GraphExecutorFactoryWrapper(std::string state) {
+    executor_factory_ = tvm_contrib_torch_decode(state.c_str());
+  }
+
+  c10::List<at::Tensor> forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    TORCH_CHECK(input_length > 0, "Receive empty list of input tensors");
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+
+    auto outputs = new DLPackTensorExt*;
+
+    auto num_outputs = tvm_contrib_torch_graph_executor_module_forward(
+        executor_factory_, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size(), outputs);

Review Comment:
   Fixed



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934081871


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }

Review Comment:
   Fixed



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r934075212


##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;
+
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+struct TVMContribTorchRuntimeModule;
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size);
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,

Review Comment:
   Fixed



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] juda commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
juda commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r933036589


##########
cmake/modules/contrib/PT_TVMDSOOP.cmake:
##########
@@ -21,38 +21,55 @@ if(NOT USE_PT_TVMDSOOP STREQUAL "OFF")
   execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch; print(torch.__path__[0].strip())"
     OUTPUT_VARIABLE PT_PATH
     RESULT_VARIABLE PT_STATUS)
-  if (NOT ${PT_STATUS} EQUAL 0)
+
+  if(NOT ${PT_STATUS} EQUAL 0)
     message(FATAL_ERROR "Fail to get pytorch path")
   endif()
 
   string(REGEX REPLACE "\n" "" PT_PATH "${PT_PATH}")
   message(STATUS "PyTorch path: ${PT_PATH}")
 
-  set(PT_COMPILE_FLAGS_STR "-I${PT_PATH}/include -D_GLIBCXX_USE_CXX11_ABI=0")
+  execute_process(COMMAND ${PYTHON_EXECUTABLE} -c "import torch;print(torch.compiled_with_cxx11_abi())"
+    OUTPUT_VARIABLE PT_CXX_FLAG
+    RESULT_VARIABLE PT_STATUS)
+
+  string(REGEX REPLACE "\n" "" PT_CXX_FLAG "${PT_CXX_FLAG}")
+  message(STATUS "Found TORCH_BUILT_WITH_CXX_ABI=${PT_CXX_FLAG} ")
+
+  if(${PT_CXX_FLAG} STREQUAL "False")
+    set(CXX_ABI_ENABLED 0)
+  else()
+    set(CXX_ABI_ENABLED 1)
+  endif()
+
+  set_property(
+    SOURCE
+    ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc
+    APPEND PROPERTY
+    COMPILE_OPTIONS
+    "-D_GLIBCXX_USE_CXX11_ABI=${CXX_ABI_ENABLED}"
+    "-I${PT_PATH}/include"
+  )
   set(PT_LINK_FLAGS_STR "-L${PT_PATH}/lib -l:libtorch.so -l:libtorch_python.so")
 
   if(NOT USE_CUDA STREQUAL "OFF")
     add_definitions(-DPT_TVMDSOOP_ENABLE_GPU)
   endif()
 
-
   string(REGEX REPLACE "\n" " " PT_FLAGS "${PT_COMPILE_FLAGS} ${PT_LINK_FLAGS}")
-  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND ${PT_COMPILE_FLAGS_STR})
+  separate_arguments(PT_COMPILE_FLAGS UNIX_COMMAND)
   separate_arguments(PT_LINK_FLAGS UNIX_COMMAND ${PT_LINK_FLAGS_STR})
 
-
   set(LIBRARY_NAME pt_tvmdsoop)
-  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/**/*.cc)
+  tvm_file_glob(GLOB_RECURSE PTTVM_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/src/contrib/torch/tvm_module_wrapper/*.cc)

Review Comment:
   Note that the compilation of `tvm_class.cc` is skipped.
   There is no way to build such a file without getting an undefined symbol error under the official PyTorch distribution.



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] yelite commented on a diff in pull request #12232: libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
yelite commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r933396360


##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;

Review Comment:
   Is this unused? Even though the `tvm_contrib_torch_delete_raw_pointer` still uses it, other `xxx_forward` functions uses`DLPackTensorExt` instead.



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};

Review Comment:
   Designated initializers are not part of the C++ 11/14 standard (https://stackoverflow.com/questions/18731707/why-does-c11-not-support-designated-initializer-lists-as-c99), so let's avoid using it



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }

Review Comment:
   It needs a deconstructor to free the `runtime_module_`



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {

Review Comment:
   Should this be placed inside namespace?



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }
+
+  void forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+    tvm_contrib_torch_operator_module_forward(
+        this->runtime_module, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size());
+
+    for (int k = 0; k < input_length; ++k) {
+      tensors[k].dl_managed_tensor->deleter(tensors[k].dl_managed_tensor);
+    }
+  }
+
+  std::string Serialize() { return std::string(tvm_contrib_torch_encode(runtime_module)); }
+
+  explicit OperatorModuleWrapper(std::string state) {
+    runtime_module = tvm_contrib_torch_decode(state.c_str());
+  }
+
+ private:
+  TVMContribTorchRuntimeModule* runtime_module;

Review Comment:
   ```suggestion
     TVMContribTorchRuntimeModule* runtime_module_;
   ```



##########
python/tvm/contrib/torch/pytorch_tvm.py:
##########
@@ -183,6 +184,11 @@ def load_tvm(self, export_dir):
 
     def build_pytorch_module(self, num_inputs, num_outputs, input_infos=None):
         """Build pytorch module containing TVM Graph Module"""
+        warnings.warn(
+            "We suggest users to use `optimized_torch` for tuning Torch modules instead",

Review Comment:
   Can you also mention this function will be removed at 0.11, giving 2 versions of buffering time between deprecation and the actual removal.



##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;
+
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+struct TVMContribTorchRuntimeModule;
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size);
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,

Review Comment:
   Consider returning a `size_t` to be consistent with others



##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;
+
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+struct TVMContribTorchRuntimeModule;
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size);
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs);

Review Comment:
   There should be a `tvm_contrib_torch_delete_dlpack_tensor_ext_array(DLPackTensorExt*)` to free the outputs.



##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+typedef DLManagedTensor** TensorList;
+
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+struct TVMContribTorchRuntimeModule;
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();

Review Comment:
   There should be a `tvm_contrib_torch_delete_runtime_module(TVMContribTorchRuntimeModule*)` to free the TVMContribTorchRuntimeModule.



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }
+
+  void forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+    tvm_contrib_torch_operator_module_forward(
+        this->runtime_module, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size());
+
+    for (int k = 0; k < input_length; ++k) {
+      tensors[k].dl_managed_tensor->deleter(tensors[k].dl_managed_tensor);
+    }
+  }
+
+  std::string Serialize() { return std::string(tvm_contrib_torch_encode(runtime_module)); }
+
+  explicit OperatorModuleWrapper(std::string state) {
+    runtime_module = tvm_contrib_torch_decode(state.c_str());
+  }
+
+ private:
+  TVMContribTorchRuntimeModule* runtime_module;
+};
+
+/**
+ * @brief A Torch's module which wraps TVM's GraphExecutorFactory Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class GraphExecutorFactoryWrapper : public torch::jit::CustomClassHolder {
+ public:
+  explicit GraphExecutorFactoryWrapper(TVMContribTorchRuntimeModule* executor_factory)
+      : executor_factory_(executor_factory) {}
+
+  GraphExecutorFactoryWrapper()
+      : GraphExecutorFactoryWrapper(tvm_contrib_torch_get_last_saved_runtime_module()) {}
+  std::string Serialize() { return tvm_contrib_torch_encode(executor_factory_); }
+
+  explicit GraphExecutorFactoryWrapper(std::string state) {
+    executor_factory_ = tvm_contrib_torch_decode(state.c_str());
+  }
+
+  c10::List<at::Tensor> forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    TORCH_CHECK(input_length > 0, "Receive empty list of input tensors");
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+
+    auto outputs = new DLPackTensorExt*;
+
+    auto num_outputs = tvm_contrib_torch_graph_executor_module_forward(
+        executor_factory_, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size(), outputs);

Review Comment:
   ```suggestion
       DLPackTensorExt* outputs;
   
       auto num_outputs = tvm_contrib_torch_graph_executor_module_forward(
           executor_factory_, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size(), &outputs);
   ```
   
   It doesn't really need to new a pointer to `DLPackTensorExt*`. A pointer to the local variable (on the stack frame) will be easier to work with. And it doesn't need to `delete outputs` later on (but it still need to delete the DLPackTensorExt array allocated inside `tvm_contrib_torch_graph_executor_module_forward`)



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);

Review Comment:
   How large is the performance penalty to create executor from executor factory every time forward is called?



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  tvm::runtime::PackedFunc run = runtime_module.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);

Review Comment:
   If the input's dtype is boolean, in `RuntimeModuleWrapperTorch.cc` it will be cast into uint8 and the `dl_tensor` here will have dtype uint8. Although the two test cases on boolean tensor pass, is this logic correct in general?



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  tvm::runtime::PackedFunc run = runtime_module.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+
+  run();
+
+  int64_t output_length = get_num_outputs();
+
+  auto outputs_ptr = new DLPackTensorExt[output_length];
+  *outputs = outputs_ptr;
+
+  for (int k = 0; k < output_length; ++k) {
+    tvm::runtime::NDArray results = get_output(k);
+    auto is_bool = results.DataType().is_bool();
+    DLManagedTensor* tensor;
+    if (is_bool) {
+      auto tmp =
+          tvm::runtime::NDArray::Empty(results.Shape(), DLDataType{kDLInt, 8, 1}, device_info);
+      results.CopyTo(tmp);
+      tensor = tmp.ToDLPack();
+    } else {
+      tensor = results.ToDLPack();
+    }

Review Comment:
   Can you extract this part as a standalone function?



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTorch.cc:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+#include <ATen/DLConvertor.h>
+#include <torch/custom_class.h>
+#include <torch/script.h>
+
+#include <iostream>
+
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+DLPackTensorExt toDLPackExt(const at::Tensor& src) {
+  if (!src.is_contiguous()) {
+    return toDLPackExt(src.contiguous());
+  }
+
+  if (src.dtype().isScalarType(torch::kBool)) {
+    auto temp = src.toType(torch::kUInt8);
+    return {.dl_managed_tensor = at::toDLPack(temp), .is_bool = true};
+  }
+
+  return {.dl_managed_tensor = at::toDLPack(src), .is_bool = false};
+}
+
+at::Tensor fromDLPackExt(const DLPackTensorExt& src) {
+  if (src.is_bool) {
+    return at::fromDLPack(src.dl_managed_tensor).toType(torch::kBool);
+  } else {
+    return at::fromDLPack(src.dl_managed_tensor);
+  }
+}
+
+/**
+ * @brief A Torch's module which wraps TVM's OperatorModule Class.
+ * The basic forward function calling TVM's runtime is provided.
+ * The TVM module can be serialized/deserialized as a Torch module.
+ */
+class OperatorModuleWrapper : public torch::jit::CustomClassHolder {
+ public:
+  OperatorModuleWrapper() { runtime_module = tvm_contrib_torch_get_last_saved_runtime_module(); }
+
+  void forward(const c10::List<at::Tensor>& inputs) {
+    int input_length = inputs.size();
+
+    std::vector<DLPackTensorExt> tensors;
+
+    for (int i = 0; i < input_length; ++i) tensors.push_back(toDLPackExt(inputs[i]));
+    tvm_contrib_torch_operator_module_forward(
+        this->runtime_module, static_cast<DLPackTensorExt*>(tensors.data()), tensors.size());

Review Comment:
   Does it really need the `static_cast` here?



##########
python/tvm/contrib/torch/pytorch_tvm.py:
##########
@@ -183,6 +184,11 @@ def load_tvm(self, export_dir):
 
     def build_pytorch_module(self, num_inputs, num_outputs, input_infos=None):
         """Build pytorch module containing TVM Graph Module"""
+        warnings.warn(
+            "We suggest users to use `optimized_torch` for tuning Torch modules instead",

Review Comment:
   Can you also add deprecation warning to the `module.py`?



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+namespace tvm {
+namespace contrib {
+
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  return {.device_type = input_device->dl_tensor.device.device_type,
+          .device_id = input_device->dl_tensor.device.device_id};
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module mod) : mod(mod) {}
+};
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+  for (int k = 0; k < input_size; ++k) {
+    setter(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+}
+
+int64_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                        DLPackTensorExt* inputs, size_t input_size,
+                                                        DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  auto device_info = tvm::contrib::getDeviceInfo(inputs[0].dl_managed_tensor);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  tvm::runtime::PackedFunc run = runtime_module.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+
+  run();
+
+  int64_t output_length = get_num_outputs();
+
+  auto outputs_ptr = new DLPackTensorExt[output_length];
+  *outputs = outputs_ptr;
+
+  for (int k = 0; k < output_length; ++k) {
+    tvm::runtime::NDArray results = get_output(k);
+    auto is_bool = results.DataType().is_bool();
+    DLManagedTensor* tensor;
+    if (is_bool) {
+      auto tmp =
+          tvm::runtime::NDArray::Empty(results.Shape(), DLDataType{kDLInt, 8, 1}, device_info);
+      results.CopyTo(tmp);
+      tensor = tmp.ToDLPack();
+    } else {
+      tensor = results.ToDLPack();
+    }
+    outputs_ptr[k] = {.dl_managed_tensor = tensor, .is_bool = is_bool};
+  }
+
+  return output_length;
+}
+
+char* tvm_contrib_torch_encode(TVMContribTorchRuntimeModule* runtime_module) {
+  auto std = tvm::contrib::serialize(runtime_module->mod);
+  auto* ret = new char[std.length() + 1];

Review Comment:
   There should be a function to delete the `char*` returned by this function



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] yelite commented on a diff in pull request #12232: [TVM PyTorch Integration] libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
yelite commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r944074708


##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+/*
+ * DLPack data structure extend with `is_bool` flag.
+ * DLPack haven't support boolean tensor,

Review Comment:
   Can you add a link to the comment, pointing to the source code of boolean tensor handling? https://github.com/pytorch/pytorch/blob/4618371da56c887195e2e1d16dad2b9686302800/aten/src/ATen/DLConvertor.cpp#L42 and https://github.com/apache/tvm/blob/de124862714e747764aa8b7f41a90bcb25f3c6a8/python/tvm/_ffi/runtime_ctypes.py#L91



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+/*
+ * Encode TVM runtime module to base64 stream
+ */
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+/*
+ * Decode TVM runtime module from base64 stream
+ */
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);  // bytes stream
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  tvm::Device ret{input_device->dl_tensor.device.device_type,
+                  input_device->dl_tensor.device.device_id};
+  return ret;
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+/*
+ * Convert NDArray to DLPack extend tensor. It should be zero-cost.
+ * @param src Pointer to NDArray
+ * @return DLPack extended tensor
+ */
+DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_bool) {
+  DLManagedTensor* tensor;
+  if (is_bool) {
+    // If we change DLDataType{kDLInt, 8, 1} to DataType::Bool()
+    // we will get `RuntimeError: Unsupported kUInt bits 1`
+    auto tmp = src->CreateView(src->Shape(), DLDataType{kDLInt, 8, 1});
+    tensor = tmp.ToDLPack();
+  } else {
+    tensor = src->ToDLPack();
+  }
+  DLPackTensorExt ret{tensor, is_bool};
+  return ret;
+}
+
+/*
+ * Create an empty NDArray with boolean type.
+ * @param src DLpack extended tensor
+ * @return an empty NDArray
+ */
+tvm::runtime::NDArray create_empty_bool_ndarray(DLPackTensorExt* src) {
+  auto& tensor = src->dl_managed_tensor->dl_tensor;
+  std::vector<int64_t> shape;
+  for (int64_t i = 0; i < tensor.ndim; i++) {
+    shape.push_back(tensor.shape[i]);
+  }
+  auto ret = tvm::runtime::NDArray::Empty(shape, DataType::Bool(), tensor.device);
+  return ret;
+}
+
+/*
+ * Create an NDArray with boolean type. (One memory copy)
+ * @param src DLpack extended tensor
+ * @return a new NDArray
+ */
+tvm::runtime::NDArray create_bool_ndarray(DLPackTensorExt* src) {
+  auto&& ret = create_empty_bool_ndarray(src);
+  ret.CopyFrom(&src->dl_managed_tensor->dl_tensor);
+  return std::move(ret);
+}
+
+bool is_zero_copy(DLPackTensorExt* src) {
+  auto& dl_tensor = src->dl_managed_tensor->dl_tensor;
+  bool is_zero_copy =
+      tvm::runtime::NDArray::AbilityOfZeroCopyForDLTensor(&dl_tensor, dl_tensor.device);
+  return is_zero_copy;
+}
+
+/*
+ * Create an NDArray from DLpack extended tensor.
+ * @param src DLpack extended tensor
+ * @return a new NDArray
+ */
+tvm::runtime::NDArray ndarray_from_dlpack(DLPackTensorExt* src) {
+  using tvm::runtime::NDArray;
+
+  NDArray array;
+  auto& dl_tensor = src->dl_managed_tensor->dl_tensor;
+  if (src->is_bool) {
+    // one memory copy
+    // the code is similar to NewFromDLTensor except for the type
+    array = create_bool_ndarray(src);
+  } else if (is_zero_copy(src)) {
+    array = NDArray::FromExternalDLTensor(src->dl_managed_tensor->dl_tensor);
+  } else {
+    // one memory copy
+    array = NDArray::NewFromDLTensor(&dl_tensor, dl_tensor.device);
+  }
+  return array;
+}
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module& mod) : mod(mod) {}
+};
+
+bool tvm_contrib_torch_is_be_copied(DLPackTensorExt* src) {
+  return (src->is_bool) || (!tvm::contrib::is_zero_copy(src));
+}
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(tvm::contrib::ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+
+  std::vector<tvm::runtime::NDArray> input_cache(input_size);
+
+  for (int k = 0; k < input_size; ++k) {
+    auto datum = tvm::contrib::ndarray_from_dlpack(&inputs[k]);  // could have one memory copy
+    input_cache[k] = datum;  // we keep the datum in a vector for future use, otherwise the datum
+                             // will be freed after the loop
+    setter(k, datum);
+  }
+
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+
+  for (int k = 0; k < input_size; ++k) {
+    if (tvm_contrib_torch_is_be_copied(&inputs[k]))
+      input_cache[k].CopyTo(&inputs[k].dl_managed_tensor->dl_tensor);
+  }
+}
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_create_graph_runtime_module(
+    TVMContribTorchRuntimeModule* graph_module, DLManagedTensor* input_example) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  tvm::Device device_info = tvm::contrib::getDeviceInfo(input_example);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  return new TVMContribTorchRuntimeModule(runtime_module);
+}
+
+size_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                                       DLPackTensorExt* inputs, size_t input_size,
+                                                       DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module->mod.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module->mod.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module->mod.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+
+  run();
+
+  int64_t output_length = get_num_outputs();
+
+  DLPackTensorExt* outputs_ptr = new DLPackTensorExt[output_length];
+  *outputs = outputs_ptr;
+
+  for (int k = 0; k < output_length; ++k) {
+    tvm::runtime::NDArray results = get_output(k);
+    bool is_bool = results.DataType().is_bool();
+    outputs_ptr[k] = tvm::contrib::create_dlpack_tensor_ext(&results, is_bool);
+  }
+
+  return output_length;
+}
+
+char* tvm_contrib_torch_encode(TVMContribTorchRuntimeModule* runtime_module) {
+  std::string std = tvm::contrib::serialize(runtime_module->mod);
+  char* ret = new char[std.length() + 1];
+  snprintf(ret, std.length() + 1, "%s", std.c_str());
+  return ret;
+}
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_decode(const char* state) {
+  tvm::runtime::Module ret = tvm::contrib::deserialize(state);
+  return new TVMContribTorchRuntimeModule(ret);
+}
+
+void tvm_contrib_torch_free_runtime_module(TVMContribTorchRuntimeModule* module_ptr) {
+  delete module_ptr;
+}
+
+void tvm_contrib_torch_free_dlpack_tensor_ext_array(DLPackTensorExt* dlpack_ptr) {
+  delete dlpack_ptr;

Review Comment:
   I think this needs `delete[]` instead of `delete`. 
   
   https://en.cppreference.com/w/cpp/language/delete
   > For the first (non-array) form, ... If expression is anything else, including if it is a pointer obtained by the array form of [new-expression](https://en.cppreference.com/w/cpp/language/new), the behavior is [undefined](https://en.cppreference.com/w/cpp/language/ub).



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+/*
+ * Encode TVM runtime module to base64 stream
+ */
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+/*
+ * Decode TVM runtime module from base64 stream
+ */
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);  // bytes stream
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  tvm::Device ret{input_device->dl_tensor.device.device_type,
+                  input_device->dl_tensor.device.device_id};
+  return ret;
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+/*
+ * Convert NDArray to DLPack extend tensor. It should be zero-cost.
+ * @param src Pointer to NDArray
+ * @return DLPack extended tensor
+ */
+DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_bool) {
+  DLManagedTensor* tensor;
+  if (is_bool) {
+    // If we change DLDataType{kDLInt, 8, 1} to DataType::Bool()
+    // we will get `RuntimeError: Unsupported kUInt bits 1`
+    auto tmp = src->CreateView(src->Shape(), DLDataType{kDLInt, 8, 1});
+    tensor = tmp.ToDLPack();
+  } else {
+    tensor = src->ToDLPack();
+  }
+  DLPackTensorExt ret{tensor, is_bool};
+  return ret;
+}
+
+/*
+ * Create an empty NDArray with boolean type.
+ * @param src DLpack extended tensor
+ * @return an empty NDArray
+ */
+tvm::runtime::NDArray create_empty_bool_ndarray(DLPackTensorExt* src) {

Review Comment:
   Maybe consider merging this function into the one below if it's only used once.



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+/*
+ * Encode TVM runtime module to base64 stream
+ */
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+/*
+ * Decode TVM runtime module from base64 stream
+ */
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);  // bytes stream
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  tvm::Device ret{input_device->dl_tensor.device.device_type,
+                  input_device->dl_tensor.device.device_id};
+  return ret;
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+/*
+ * Convert NDArray to DLPack extend tensor. It should be zero-cost.
+ * @param src Pointer to NDArray
+ * @return DLPack extended tensor
+ */
+DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_bool) {

Review Comment:
   Can you make sure all C++ functions follow the camel case naming?



##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+/*
+ * DLPack data structure extend with `is_bool` flag.
+ * DLPack haven't support boolean tensor,
+ * thus a boolean tensor will be regarded as a UInt8 tensor.
+ */
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+/*
+ * A wrapper pointing to TVM runtime module.
+ */
+struct TVMContribTorchRuntimeModule;
+
+/*
+ * Obtain a saved runtime module passed by TVM FFI.
+ * @return A TVM runtime module wrapper.
+ */
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();
+
+/*
+ * Delete TVMContribTorchRuntimeModule pointer.
+ */
+void tvm_contrib_torch_free_runtime_module(TVMContribTorchRuntimeModule* module_ptr);
+
+/*
+ * Obtain ExecutorFactory runtime module from ExecutorFactory class.
+ * @param graph_module ExecutorFactory class
+ * @param input_example For obtaining device information
+ * @return ExecutorFactory TVM runtime module wrapper
+ */
+TVMContribTorchRuntimeModule* tvm_contrib_torch_create_graph_runtime_module(
+    TVMContribTorchRuntimeModule* graph_module, DLManagedTensor* input_example);

Review Comment:
   ```suggestion
       TVMContribTorchRuntimeModule* graph_executor_factory, DLManagedTensor* input_example);
   ```



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+/*
+ * Encode TVM runtime module to base64 stream
+ */
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+/*
+ * Decode TVM runtime module from base64 stream
+ */
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);  // bytes stream
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {

Review Comment:
   tvm::Device is an alias to DLDevice. Why do we want to do this instead of using `input_device->dl_tensor.device`? Also just noticed the variable name doesn't look correct...



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+/*
+ * Encode TVM runtime module to base64 stream
+ */
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+/*
+ * Decode TVM runtime module from base64 stream
+ */
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);  // bytes stream
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  tvm::Device ret{input_device->dl_tensor.device.device_type,
+                  input_device->dl_tensor.device.device_id};
+  return ret;
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+/*
+ * Convert NDArray to DLPack extend tensor. It should be zero-cost.
+ * @param src Pointer to NDArray
+ * @return DLPack extended tensor
+ */
+DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_bool) {

Review Comment:
   Can it get the `is_bool` inside this function by looking at `src`? If yes we can remove this parameter



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.
+ */
+#include <dlpack/dlpack.h>
+#include <dmlc/memory_io.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/target/codegen.h>
+#include <tvm/target/target.h>
+
+#include <cstdio>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "../../../runtime/graph_executor/graph_executor_factory.h"
+#include "../base64.h"
+#include "runtime_bridge.h"
+
+namespace tvm {
+namespace contrib {
+
+struct ThreadLocalStore {
+  tvm::runtime::Module mod;
+  static ThreadLocalStore* ThreadLocal() {
+    thread_local ThreadLocalStore tls;
+    return &tls;
+  }
+};
+
+/*
+ * Encode TVM runtime module to base64 stream
+ */
+std::string serialize(tvm::runtime::Module module) {
+  static const runtime::PackedFunc* f_to_str =
+      runtime::Registry::Get("script_torch.save_to_base64");
+  ICHECK(f_to_str) << "IndexError: Cannot find the packed function "
+                      "`script_torch.save_to_base64` in the global registry";
+  return (*f_to_str)(module);
+}
+
+struct Deleter {  // deleter
+  explicit Deleter(std::string file_name) { this->file_name = file_name; }
+  void operator()(FILE* p) const {
+    fclose(p);
+    ICHECK(remove(file_name.c_str()) == 0)
+        << "remove temporary file (" << file_name << ") unsuccessfully";
+  }
+  std::string file_name;
+};
+
+/*
+ * Decode TVM runtime module from base64 stream
+ */
+tvm::runtime::Module deserialize(std::string state) {
+  auto length = tvm::support::b64strlen(state);
+
+  std::vector<u_char> bytes(length);  // bytes stream
+  tvm::support::b64decode(state, bytes.data());
+
+  const std::string name = tmpnam(NULL);
+  auto file_name = name + ".so";
+  std::unique_ptr<FILE, Deleter> pFile(fopen(file_name.c_str(), "wb"), Deleter(file_name));
+  fwrite(bytes.data(), sizeof(u_char), length, pFile.get());
+  fflush(pFile.get());
+
+  std::string load_f_name = "runtime.module.loadfile_so";
+  const PackedFunc* f = runtime::Registry::Get(load_f_name);
+  ICHECK(f != nullptr) << "Loader for `.so` files is not registered,"
+                       << " resolved to (" << load_f_name << ") in the global registry."
+                       << "Ensure that you have loaded the correct runtime code, and"
+                       << "that you are on the correct hardware architecture.";
+
+  tvm::runtime::Module ret = (*f)(file_name, "");
+
+  return ret;
+}
+
+tvm::Device getDeviceInfo(DLManagedTensor* input_device) {
+  tvm::Device ret{input_device->dl_tensor.device.device_type,
+                  input_device->dl_tensor.device.device_id};
+  return ret;
+}
+
+TVM_REGISTER_GLOBAL("tvmtorch.save_runtime_mod").set_body_typed([](tvm::runtime::Module mod) {
+  ThreadLocalStore::ThreadLocal()->mod = mod;
+});
+
+/*
+ * Convert NDArray to DLPack extend tensor. It should be zero-cost.
+ * @param src Pointer to NDArray
+ * @return DLPack extended tensor
+ */
+DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_bool) {
+  DLManagedTensor* tensor;
+  if (is_bool) {
+    // If we change DLDataType{kDLInt, 8, 1} to DataType::Bool()
+    // we will get `RuntimeError: Unsupported kUInt bits 1`
+    auto tmp = src->CreateView(src->Shape(), DLDataType{kDLInt, 8, 1});
+    tensor = tmp.ToDLPack();
+  } else {
+    tensor = src->ToDLPack();
+  }
+  DLPackTensorExt ret{tensor, is_bool};
+  return ret;
+}
+
+/*
+ * Create an empty NDArray with boolean type.
+ * @param src DLpack extended tensor
+ * @return an empty NDArray
+ */
+tvm::runtime::NDArray create_empty_bool_ndarray(DLPackTensorExt* src) {
+  auto& tensor = src->dl_managed_tensor->dl_tensor;
+  std::vector<int64_t> shape;
+  for (int64_t i = 0; i < tensor.ndim; i++) {
+    shape.push_back(tensor.shape[i]);
+  }
+  auto ret = tvm::runtime::NDArray::Empty(shape, DataType::Bool(), tensor.device);
+  return ret;
+}
+
+/*
+ * Create an NDArray with boolean type. (One memory copy)
+ * @param src DLpack extended tensor
+ * @return a new NDArray
+ */
+tvm::runtime::NDArray create_bool_ndarray(DLPackTensorExt* src) {
+  auto&& ret = create_empty_bool_ndarray(src);
+  ret.CopyFrom(&src->dl_managed_tensor->dl_tensor);
+  return std::move(ret);
+}
+
+bool is_zero_copy(DLPackTensorExt* src) {
+  auto& dl_tensor = src->dl_managed_tensor->dl_tensor;
+  bool is_zero_copy =
+      tvm::runtime::NDArray::AbilityOfZeroCopyForDLTensor(&dl_tensor, dl_tensor.device);
+  return is_zero_copy;
+}
+
+/*
+ * Create an NDArray from DLpack extended tensor.
+ * @param src DLpack extended tensor
+ * @return a new NDArray
+ */
+tvm::runtime::NDArray ndarray_from_dlpack(DLPackTensorExt* src) {
+  using tvm::runtime::NDArray;
+
+  NDArray array;
+  auto& dl_tensor = src->dl_managed_tensor->dl_tensor;
+  if (src->is_bool) {
+    // one memory copy
+    // the code is similar to NewFromDLTensor except for the type
+    array = create_bool_ndarray(src);
+  } else if (is_zero_copy(src)) {
+    array = NDArray::FromExternalDLTensor(src->dl_managed_tensor->dl_tensor);
+  } else {
+    // one memory copy
+    array = NDArray::NewFromDLTensor(&dl_tensor, dl_tensor.device);
+  }
+  return array;
+}
+
+}  // namespace contrib
+}  // namespace tvm
+
+extern "C" {
+
+struct TVMContribTorchRuntimeModule {
+  tvm::runtime::Module mod;
+
+  explicit TVMContribTorchRuntimeModule(tvm::runtime::Module& mod) : mod(mod) {}
+};
+
+bool tvm_contrib_torch_is_be_copied(DLPackTensorExt* src) {
+  return (src->is_bool) || (!tvm::contrib::is_zero_copy(src));
+}
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module() {
+  return new TVMContribTorchRuntimeModule(tvm::contrib::ThreadLocalStore::ThreadLocal()->mod);
+}
+
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("__tvm_main__");
+
+  std::vector<TVMValue> tvm_values(input_size);
+  std::vector<int> tvm_type_codes(input_size);
+  tvm::runtime::TVMArgsSetter setter(tvm_values.data(), tvm_type_codes.data());
+
+  std::vector<tvm::runtime::NDArray> input_cache(input_size);
+
+  for (int k = 0; k < input_size; ++k) {
+    auto datum = tvm::contrib::ndarray_from_dlpack(&inputs[k]);  // could have one memory copy
+    input_cache[k] = datum;  // we keep the datum in a vector for future use, otherwise the datum
+                             // will be freed after the loop
+    setter(k, datum);
+  }
+
+  run.CallPacked(tvm::runtime::TVMArgs(tvm_values.data(), tvm_type_codes.data(), input_size),
+                 nullptr);
+
+  for (int k = 0; k < input_size; ++k) {
+    if (tvm_contrib_torch_is_be_copied(&inputs[k]))
+      input_cache[k].CopyTo(&inputs[k].dl_managed_tensor->dl_tensor);
+  }
+}
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_create_graph_runtime_module(
+    TVMContribTorchRuntimeModule* graph_module, DLManagedTensor* input_example) {
+  tvm::runtime::PackedFunc built_module = graph_module->mod.GetFunction("default");
+  tvm::Device device_info = tvm::contrib::getDeviceInfo(input_example);
+  tvm::runtime::Module runtime_module = built_module(device_info);
+  return new TVMContribTorchRuntimeModule(runtime_module);
+}
+
+size_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                                       DLPackTensorExt* inputs, size_t input_size,
+                                                       DLPackTensorExt** outputs) {
+  tvm::runtime::PackedFunc run = runtime_module->mod.GetFunction("run");
+  tvm::runtime::PackedFunc set_input = runtime_module->mod.GetFunction("set_input");
+  tvm::runtime::PackedFunc get_output = runtime_module->mod.GetFunction("get_output");
+  tvm::runtime::PackedFunc get_num_outputs = runtime_module->mod.GetFunction("get_num_outputs");
+
+  for (int k = 0; k < input_size; ++k) {
+    set_input(k, &inputs[k].dl_managed_tensor->dl_tensor);
+  }
+
+  run();
+
+  int64_t output_length = get_num_outputs();
+
+  DLPackTensorExt* outputs_ptr = new DLPackTensorExt[output_length];
+  *outputs = outputs_ptr;
+
+  for (int k = 0; k < output_length; ++k) {
+    tvm::runtime::NDArray results = get_output(k);
+    bool is_bool = results.DataType().is_bool();
+    outputs_ptr[k] = tvm::contrib::create_dlpack_tensor_ext(&results, is_bool);
+  }
+
+  return output_length;
+}
+
+char* tvm_contrib_torch_encode(TVMContribTorchRuntimeModule* runtime_module) {
+  std::string std = tvm::contrib::serialize(runtime_module->mod);
+  char* ret = new char[std.length() + 1];
+  snprintf(ret, std.length() + 1, "%s", std.c_str());
+  return ret;
+}
+
+TVMContribTorchRuntimeModule* tvm_contrib_torch_decode(const char* state) {
+  tvm::runtime::Module ret = tvm::contrib::deserialize(state);
+  return new TVMContribTorchRuntimeModule(ret);
+}
+
+void tvm_contrib_torch_free_runtime_module(TVMContribTorchRuntimeModule* module_ptr) {
+  delete module_ptr;
+}
+
+void tvm_contrib_torch_free_dlpack_tensor_ext_array(DLPackTensorExt* dlpack_ptr) {
+  delete dlpack_ptr;
+}
+
+void tvm_contrib_torch_free_encoding(char* encoding) { delete encoding; }

Review Comment:
   Same problem on `delete[]` vs `delete`



##########
src/contrib/torch/tvm_module_wrapper/runtime_bridge.h:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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 runtime_bridge.h
+ * \brief Util functions for pytorch tvm interaction.
+ */
+#ifndef TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+#define TVM_CONTRIB_TORCH_TVM_MODULE_WRAPPER_RUNTIME_BRIDGE_H_
+
+extern "C" {
+
+/*
+ * DLPack data structure extend with `is_bool` flag.
+ * DLPack haven't support boolean tensor,
+ * thus a boolean tensor will be regarded as a UInt8 tensor.
+ */
+struct DLPackTensorExt {
+  DLManagedTensor* dl_managed_tensor;
+  bool is_bool;
+};
+
+/*
+ * A wrapper pointing to TVM runtime module.
+ */
+struct TVMContribTorchRuntimeModule;
+
+/*
+ * Obtain a saved runtime module passed by TVM FFI.
+ * @return A TVM runtime module wrapper.
+ */
+TVMContribTorchRuntimeModule* tvm_contrib_torch_get_last_saved_runtime_module();
+
+/*
+ * Delete TVMContribTorchRuntimeModule pointer.
+ */
+void tvm_contrib_torch_free_runtime_module(TVMContribTorchRuntimeModule* module_ptr);
+
+/*
+ * Obtain ExecutorFactory runtime module from ExecutorFactory class.
+ * @param graph_module ExecutorFactory class
+ * @param input_example For obtaining device information
+ * @return ExecutorFactory TVM runtime module wrapper
+ */
+TVMContribTorchRuntimeModule* tvm_contrib_torch_create_graph_runtime_module(
+    TVMContribTorchRuntimeModule* graph_module, DLManagedTensor* input_example);
+
+/*
+ * Forward method for OperatorModuleWrapper.
+ * @param runtime_module TVM runtime module wrapper
+ * @param inputs Array pointer of the input tensors
+ * @param input_size The number of input tensors
+ */
+void tvm_contrib_torch_operator_module_forward(TVMContribTorchRuntimeModule* runtime_module,
+                                               DLPackTensorExt* inputs, size_t input_size);
+
+/*
+ * Forward method for GraphExecutorFactoryWrapper.
+ * @param graph_module TVM runtime module wrapper
+ * @param inputs Array pointer of the input tensors
+ * @param input_size The number of input tensors
+ * @param outputs The resulting output tensors pointer
+ * @return The number of output tensors
+ */
+size_t tvm_contrib_torch_graph_executor_module_forward(TVMContribTorchRuntimeModule* graph_module,
+                                                       DLPackTensorExt* inputs, size_t input_size,
+                                                       DLPackTensorExt** outputs);
+
+/*
+ * Encode TVM runtime module.
+ * @param runtime_module TVM runtime module wrapper
+ * @return The encoding stream (char array)
+ */
+char* tvm_contrib_torch_encode(TVMContribTorchRuntimeModule* runtime_module);
+
+/*
+ * Decode TVM runtime module.
+ * @param state The encoding stream (char array) of TVM runtime module
+ * @return TVM runtime module wrapper
+ */
+TVMContribTorchRuntimeModule* tvm_contrib_torch_decode(const char* state);
+
+/*
+ * Delete DLPackTensorExt pointer.
+ */
+void tvm_contrib_torch_free_dlpack_tensor_ext_array(DLPackTensorExt*);
+
+/*
+ * Delete char array pointer.
+ */
+void tvm_contrib_torch_free_encoding(char* encoding);
+
+/*
+ * Checking if a DLPackTensorExt is boolean or cannot be copied in zero cost.
+ */
+bool tvm_contrib_torch_is_be_copied(DLPackTensorExt*);

Review Comment:
   ```suggestion
   bool tvm_contrib_torch_tensor_ability_of_zero_copy(DLPackTensorExt*);
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] yelite commented on a diff in pull request #12232: [TVM PyTorch Integration] libstdc++ CXX11 ABI Compatibility & boolean tensor support

Posted by GitBox <gi...@apache.org>.
yelite commented on code in PR #12232:
URL: https://github.com/apache/tvm/pull/12232#discussion_r946776348


##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -121,53 +116,41 @@ DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_boo
 }
 
 /*
- * Create an empty NDArray with boolean type.
+ * Create an NDArray with boolean type. (One memory copy)
  * @param src DLpack extended tensor
- * @return an empty NDArray
+ * @return a new NDArray
  */
-tvm::runtime::NDArray create_empty_bool_ndarray(DLPackTensorExt* src) {
+tvm::runtime::NDArray createBoolNDarray(DLPackTensorExt* src) {
   auto& tensor = src->dl_managed_tensor->dl_tensor;
   std::vector<int64_t> shape;
   for (int64_t i = 0; i < tensor.ndim; i++) {
     shape.push_back(tensor.shape[i]);
   }
   auto ret = tvm::runtime::NDArray::Empty(shape, DataType::Bool(), tensor.device);
-  return ret;
-}
-
-/*
- * Create an NDArray with boolean type. (One memory copy)
- * @param src DLpack extended tensor
- * @return a new NDArray
- */
-tvm::runtime::NDArray create_bool_ndarray(DLPackTensorExt* src) {
-  auto&& ret = create_empty_bool_ndarray(src);
   ret.CopyFrom(&src->dl_managed_tensor->dl_tensor);
   return std::move(ret);
 }
 
-bool is_zero_copy(DLPackTensorExt* src) {
+bool isZeroCopy(DLPackTensorExt* src) {
   auto& dl_tensor = src->dl_managed_tensor->dl_tensor;
-  bool is_zero_copy =
-      tvm::runtime::NDArray::AbilityOfZeroCopyForDLTensor(&dl_tensor, dl_tensor.device);
-  return is_zero_copy;
+  return tvm::runtime::NDArray::AbilityOfZeroCopyForDLTensor(&dl_tensor, dl_tensor.device);
 }
 
 /*
  * Create an NDArray from DLpack extended tensor.
  * @param src DLpack extended tensor
  * @return a new NDArray
  */
-tvm::runtime::NDArray ndarray_from_dlpack(DLPackTensorExt* src) {
+tvm::runtime::NDArray ndarrayFromDLpack(DLPackTensorExt* src) {

Review Comment:
   ```suggestion
   tvm::runtime::NDArray NDArrayFromDLpack(DLPackTensorExt* src) {
   ```



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -121,53 +116,41 @@ DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_boo
 }
 
 /*
- * Create an empty NDArray with boolean type.
+ * Create an NDArray with boolean type. (One memory copy)
  * @param src DLpack extended tensor
- * @return an empty NDArray
+ * @return a new NDArray
  */
-tvm::runtime::NDArray create_empty_bool_ndarray(DLPackTensorExt* src) {
+tvm::runtime::NDArray createBoolNDarray(DLPackTensorExt* src) {

Review Comment:
   ```suggestion
   tvm::runtime::NDArray CreateBoolNDArray(DLPackTensorExt* src) {
   ```



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -187,8 +170,8 @@ struct TVMContribTorchRuntimeModule {
   explicit TVMContribTorchRuntimeModule(tvm::runtime::Module& mod) : mod(mod) {}
 };
 
-bool tvm_contrib_torch_is_be_copied(DLPackTensorExt* src) {
-  return (src->is_bool) || (!tvm::contrib::is_zero_copy(src));
+bool tvm_contrib_torch_tensor_ability_of_zero_copy(DLPackTensorExt* src) {

Review Comment:
   The function name sounds like it will return True if it's able to zero copy the tensor but it actually returns False if it's able to zero copy. Can you either update the function body or function name to make it consistent



##########
src/contrib/torch/tvm_module_wrapper/RuntimeModuleWrapperTVM.cc:
##########
@@ -121,53 +116,41 @@ DLPackTensorExt create_dlpack_tensor_ext(tvm::runtime::NDArray* src, bool is_boo
 }
 
 /*
- * Create an empty NDArray with boolean type.
+ * Create an NDArray with boolean type. (One memory copy)
  * @param src DLpack extended tensor
- * @return an empty NDArray
+ * @return a new NDArray
  */
-tvm::runtime::NDArray create_empty_bool_ndarray(DLPackTensorExt* src) {
+tvm::runtime::NDArray createBoolNDarray(DLPackTensorExt* src) {
   auto& tensor = src->dl_managed_tensor->dl_tensor;
   std::vector<int64_t> shape;
   for (int64_t i = 0; i < tensor.ndim; i++) {
     shape.push_back(tensor.shape[i]);
   }
   auto ret = tvm::runtime::NDArray::Empty(shape, DataType::Bool(), tensor.device);
-  return ret;
-}
-
-/*
- * Create an NDArray with boolean type. (One memory copy)
- * @param src DLpack extended tensor
- * @return a new NDArray
- */
-tvm::runtime::NDArray create_bool_ndarray(DLPackTensorExt* src) {
-  auto&& ret = create_empty_bool_ndarray(src);
   ret.CopyFrom(&src->dl_managed_tensor->dl_tensor);
   return std::move(ret);
 }
 
-bool is_zero_copy(DLPackTensorExt* src) {
+bool isZeroCopy(DLPackTensorExt* src) {

Review Comment:
   ```suggestion
   bool IsZeroCopy(DLPackTensorExt* src) {
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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