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/02/17 03:25:33 UTC

[GitHub] [tvm] areusch opened a new pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   This PR adds the AOTExecutor implementation and the metadata code-generator for use with the `c` target. This PR allows users to run models which use only the CPU via the AOTExecutor using the C++ runtime.
   
   TODO: fix ci
   TODO: remove cast hack
   TODO: add additional tests
   TODO: documentation
   
   cc @Mousius @manupa-arm @kparzysz-quic @masahi @mehrdadh


-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: python/tvm/runtime/executor/__init__.py
##########
@@ -0,0 +1,26 @@
+# 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.
+
+"""This module contains Python wrappers for the TVM C++ Executor implementations.
+
+NOTE: at present, only AOT Executor is contained here. The others are:
+ - GraphExecutor, in python/tvm/contrib/graph_executor.py
+ - VM Executor, in python/tvm/runtime/vm.py
+
+TODO(areusch): Consolidate these into this module.

Review comment:
       I just realized that, we have two notions of `executor`. One is the runtime one above, the other is 
   https://github.com/apache/tvm/blob/55849e651e3dff6cffeb7fd14aa89699e9575d10/python/tvm/relay/build_module.py#L647
   which is used a lot in the test cases.
   
   Do we intend to support `create_executor(kind="aot", ...)`, given that we can now run things via the cpp runtime?




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

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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: include/tvm/runtime/metadata.h
##########
@@ -75,6 +75,13 @@ struct TVMMetadata {
   const struct TVMTensorInfo* outputs;
   /*! \brief Number of elements in `outputs` array. */
   int64_t num_outputs;
+  /*! \brief Memory Pools needed by the AOT run_model function.

Review comment:
       That is right!.
   
   We had run (entry_point), run_model and  __tvm_main__.
   The entry_point is supposed to call run_model, however, run_model is identical to tvm_main -- therefore, it was removed due to no need of maintaining two symbols for relay and tir versions of main.
   
   So I think it needs to be tvm_main now.




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

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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -177,6 +179,12 @@ class AOTOnDemandAllocator : public transform::DeviceAwareExprVisitor {
       for (auto sid : sinfo->storage_ids) {
         return_ids_.push_back(sid);
       }
+      return_ttypes_.clear();
+      auto ttypes = FlattenTupleType(e->checked_type());
+      return_ttypes_.reserve(ttypes.size());
+      for (auto ttype : ttypes) {
+        return_ttypes_.push_back(ttype);
+      }

Review comment:
       cc @areusch 




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

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

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



[GitHub] [tvm] ekalda commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   Yeah the Ethos-U test have been really flaky lately, there is always a random assortment failing and a random assortment has been disabled. By the looks of it, there is something wrong with running the FVP, @lhutton1 has been looking into it, but we can't reproduce it locally, so we would need to progress https://github.com/apache/tvm/pull/10214 to get a somewhat more meaningful errors in upstream CI...


-- 
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] ekalda edited a comment on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   Yeah the Ethos-U tests have been really flaky lately, there is always a random assortment failing and a random assortment has been disabled. By the looks of it, there is something wrong with running the FVP, @lhutton1 has been looking into it, but we can't reproduce it locally, so we would need to progress https://github.com/apache/tvm/pull/10214 to get a somewhat more meaningful errors in upstream CI...


-- 
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] manupa-arm commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -317,6 +327,16 @@ class AOTExecutorCodegen : public MixedModeVisitor {
     }
   }
 
+  void PushArgs(const Expr& expr, std::vector<tir::Var> sids, Array<PrimExpr>* args) {

Review comment:
       nit : any reason mixed pass-by-value, pass-by-(const)-references usage and pointers here ?

##########
File path: python/tvm/runtime/executor/aot_executor.py
##########
@@ -0,0 +1,182 @@
+# 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.
+"""A Python wrapper for the Module-based Model Runtime Interface for Ahead-of-Time compilation."""
+
+import numpy as np
+
+
+class AotModule(object):

Review comment:
       (stylistic) : AotModule --> AOTModule or AoTModule ?

##########
File path: tests/python/relay/aot/test_crt_aot.py
##########
@@ -49,7 +50,14 @@ def test_error_c_interface_with_packed_api():
     two = relay.add(relay.const(1), relay.const(1))
     func = relay.Function([], two)
 
-    with pytest.raises(tvm.TVMError, match="Packed interface required for packed operators"):
+    with pytest.raises(
+        tvm.TVMError,
+        match=re.escape(

Review comment:
       From the above comment, I'd expect two of these tests here. 

##########
File path: src/runtime/aot_executor/aot_executor_factory.cc
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 aot_executor_factory.cc
+ * \brief Graph executor factory implementations
+ */
+
+#include "./aot_executor_factory.h"
+
+#include <tvm/runtime/container/string.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+
+#include <iterator>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+AotExecutorFactory::AotExecutorFactory(

Review comment:
       Just for my knowledge, I never understood why these are called "factories". What maybe the reason ?  :) 

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -857,11 +905,30 @@ class AOTExecutorCodegen : public MixedModeVisitor {
     ICHECK(target_host_.defined()) << "require a target_host to be given for AOT codegen";
     VLOG(1) << "target host: " << target_host_->ToDebugString();
 
+    Runtime runtime_config = mod->GetAttr<Runtime>(tvm::attr::kRuntime).value();
     Executor executor_config = mod->GetAttr<Executor>(tvm::attr::kExecutor).value();
     String interface_api = executor_config->GetAttr<String>("interface-api").value_or("packed");
     Integer workspace_byte_alignment =
         executor_config->GetAttr<Integer>("workspace-byte-alignment").value_or(16);
     use_unpacked_api_ = executor_config->GetAttr<Bool>("unpacked-api").value_or(Bool(false));
+    use_call_cpacked_ = !use_unpacked_api_;
+
+    // Validate choice of use_unpacked_api_ and use_call_cpacked_
+    if (runtime_config->name == kTvmRuntimeCrt) {
+      CHECK(interface_api == "packed" || static_cast<bool>(use_unpacked_api_) == true)

Review comment:
       Are both of these CHECKs tested ? 

##########
File path: src/runtime/aot_executor/aot_executor_factory.cc
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 aot_executor_factory.cc
+ * \brief Graph executor factory implementations

Review comment:
       Need update




-- 
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] tmoreau89 removed a comment on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

Posted by GitBox <gi...@apache.org>.
tmoreau89 removed a comment on pull request #10283:
URL: https://github.com/apache/tvm/pull/10283#issuecomment-1058263983


   @manupa-arm the PR is now ready to be re-reviewed, thank you!


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

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

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



[GitHub] [tvm] manupa-arm commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   Seems like some u65 tests are failing. Please can you file an issue?
   
   cc : @ekalda 


-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: python/tvm/runtime/executor/aot_executor.py
##########
@@ -0,0 +1,182 @@
+# 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.
+"""A Python wrapper for the Module-based Model Runtime Interface for Ahead-of-Time compilation."""
+
+import numpy as np
+
+
+class AotModule(object):
+    """Wraps the AOT executor runtime.Module.
+
+    This is a thin wrapper of the underlying TVM module.
+    you can also directly call set_input, run, and get_output
+    of underlying module functions
+
+    Parameters
+    ----------
+    module : tvm.runtime.Module
+        The internal tvm module that holds the actual graph functions.
+
+    Attributes
+    ----------
+    module : tvm.runtime.Module
+        The internal tvm module that holds the actual graph functions.
+
+    Examples
+    --------
+
+    .. code-block:: python
+
+        import tvm
+        from tvm import relay
+        from tvm.contrib import graph_executor
+
+        # build the library using graph executor
+        lib = relay.build(...)
+        lib.export_library("compiled_lib.so")
+        # load it back as a runtime
+        lib: tvm.runtime.Module = tvm.runtime.load_module("compiled_lib.so")
+        # Call the library factory function for default and create
+        # a new runtime.Module, wrap with graph module.
+        gmod = graph_executor.GraphModule(lib["default"](dev))

Review comment:
       Need update?




-- 
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] areusch commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   if we see more of the same flakiness with similar traceback we can add it to #10300; I'm not convinced it's the same thing here just yet (since other unittests were failing and that was due to a divergence in the way `tvm.build` and `tvm.relay.build` populate IRModule attrs (see #10373 ).


-- 
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] tmoreau89 merged pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   


-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/runtime/metadata.cc
##########
@@ -52,5 +62,65 @@ TensorInfo::TensorInfo(const struct ::TVMTensorInfo* data)
 TVM_REGISTER_OBJECT_TYPE(TensorInfoNode);
 
 }  // namespace metadata
+
+class MetadataModuleNode : public ::tvm::runtime::ModuleNode {
+ public:
+  explicit MetadataModuleNode(runtime::metadata::Metadata metadata) {
+    // CHECK((metadata.defined() && code.size() > 0) || (!metadata.defined() && code.size() == 0))
+    //   << "metadata and code must both be either defined (when passed from compiler) or undefined
+    //   "
+    //   << "(when passed from runtime)";
+    metadata_ = metadata;
+    //    code_ = code;
+  }
+
+  const char* type_key() const { return "metadata_module"; }
+
+  static Module LoadFromBinary() {
+    return Module(make_object<MetadataModuleNode>(runtime::metadata::Metadata()));
+  }
+
+  void SaveToBinary(dmlc::Stream* stream) final {}
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_metadata") {
+      return PackedFunc([this, sptr_to_self](TVMArgs args, TVMRetValue* rv) {
+        if (!metadata_.defined()) {
+          TVMFunctionHandle f_handle;
+          int32_t ret_code = TVMBackendGetFuncFromEnv(this, "get_c_metadata", &f_handle);

Review comment:
       See https://github.com/apache/tvm/pull/10283/files#r817105544




-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/runtime/aot_executor/aot_executor.h
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.h
+ */
+#ifndef TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+#define TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+
+#include <tvm/runtime/metadata.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+
+#include <string>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+class TVM_DLL AotExecutor : public ModuleNode {
+ public:
+  /*!
+   * \brief Implements member function lookup for this Module for the frontend.
+   * \param name The name of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The corresponding member function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override;
+
+  /*!
+   * \return The type key of the executor.
+   */
+  const char* type_key() const final { return "AotExecutor"; }
+
+  void Run();
+
+  /*!
+   * \brief Initialize the AOT executor with metadata, runtime::Module, and device.
+   * \param module The module containing the compiled functions for the host
+   *  processor.
+   * \param devs The device of the host and devices where graph nodes will be

Review comment:
       There is no "graph".
   
   And why only the first element of `devs` is ever used in `aot_executor.cc`?




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

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 commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   No this one https://ci.tlcpack.ai/blue/organizations/jenkins/tvm/detail/PR-10283/17/pipeline/307


-- 
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] tmoreau89 commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   Thank you @areusch @masahi @kparzysz-quic @manupa-arm for the reviews! The PR has been merged.


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

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 commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   I think there is already https://github.com/apache/tvm/issues/10300 but not sure if it is the same one.
   
   The point is we have many flaky ethos-u issues going on lately.


-- 
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] tmoreau89 commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   @manupa-arm the PR is now ready to be re-reviewed, thank you!


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

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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: python/tvm/runtime/executor/__init__.py
##########
@@ -0,0 +1,26 @@
+# 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.
+
+"""This module contains Python wrappers for the TVM C++ Executor implementations.
+
+NOTE: at present, only AOT Executor is contained here. The others are:
+ - GraphExecutor, in python/tvm/contrib/graph_executor.py
+ - VM Executor, in python/tvm/runtime/vm.py
+
+TODO(areusch): Consolidate these into this module.

Review comment:
       I just realized that, we have two notions of `executor`. One is runtime one above, the other is 
   https://github.com/apache/tvm/blob/55849e651e3dff6cffeb7fd14aa89699e9575d10/python/tvm/relay/build_module.py#L647
   
   Do we intend to support `create_executor(kind="aot", ...)`?




-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/target/metadata_module.cc
##########
@@ -25,17 +25,145 @@
 
 #include <tvm/relay/runtime.h>
 
+#include <utility>
 #include <vector>
 
+#include "../runtime/const_loader_module.h"
 #include "../runtime/meta_data.h"
 #include "llvm/llvm_module.h"
 #include "source/source_module.h"
 
 namespace tvm {
 namespace codegen {
 
+static runtime::Module CreateCrtMetadataModule(
+    runtime::Module target_module, Target target, relay::Runtime runtime,
+    relay::backend::ExecutorCodegenMetadata metadata,
+    Array<runtime::Module> non_crt_exportable_modules,
+    Array<runtime::Module> crt_exportable_modules,
+    const std::unordered_map<std::string, runtime::NDArray>& const_var_ndarray) {
+  if (!non_crt_exportable_modules.empty()) {
+    std::string non_exportable_modules;
+    for (unsigned int i = 0; i < non_crt_exportable_modules.size(); i++) {
+      if (i > 0) {
+        non_exportable_modules += ", ";
+      }
+      auto mod = non_crt_exportable_modules[i];
+      auto pf_sym = mod.GetFunction("get_symbol");
+      if (pf_sym != nullptr) {
+        non_exportable_modules += pf_sym().operator std::string();
+      } else {
+        non_exportable_modules +=
+            std::string{"(module type_key="} + mod->type_key() + std::string{")"};
+      }
+    }
+    CHECK(false) << "These " << non_crt_exportable_modules.size()
+                 << " modules are not exportable to C-runtime: " << non_exportable_modules;
+  }
+
+  if (target->kind->name == "c") {
+    crt_exportable_modules.push_back(target_module);
+    target_module =
+        CreateCSourceCrtMetadataModule(crt_exportable_modules, target, runtime, metadata);
+  } else if (target->kind->name == "llvm") {
+#ifdef TVM_LLVM_VERSION
+    crt_exportable_modules.push_back(target_module);
+    target_module = CreateLLVMCrtMetadataModule(crt_exportable_modules, target, runtime);
+#else   // TVM_LLVM_VERSION
+    LOG(FATAL) << "TVM was not built with LLVM enabled.";
+#endif  // TVM_LLVM_VERSION
+  }
+
+  return target_module;
+}
+
+// TODO(areusch,masahi): Unify metadata representation and remove the need for this function
+static runtime::metadata::Metadata ConvertMetaData(
+    relay::backend::ExecutorCodegenMetadata metadata) {
+  std::vector<runtime::metadata::TensorInfo> inputs;
+  ICHECK(metadata.defined());
+  for (size_t i = 0; i < metadata->inputs.size(); ++i) {
+    auto v = metadata->inputs[i];
+    auto ttype = metadata->input_tensor_types[i];
+    inputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            v->name_hint, relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> outputs;
+  auto output_ttypes = metadata->output_tensor_types;
+  for (size_t i = 0; i < output_ttypes.size(); ++i) {
+    auto ttype = output_ttypes[i];
+    std::stringstream name;
+    name << "output" << i;
+    outputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            name.str(), relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> pools;
+  for (size_t i = 0; i < metadata->pools.size(); ++i) {
+    auto var = metadata->pools[i];
+    pools.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            var->name_hint,
+            std::vector<int64_t>{metadata->pool_inputs.value()[var]->allocated_size},

Review comment:
       Check that `metadata->pool_inputs` is non-null (it's `Optional<Map<tir::Var, tir::usmp::AllocatedPoolInfo>>`)




-- 
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] areusch commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: tests/python/relay/aot/test_cpp_aot.py
##########
@@ -0,0 +1,118 @@
+# 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.
+
+
+import sys
+import textwrap
+
+import numpy as np
+import pytest
+
+import tvm
+from tvm import relay, TVMError
+from tvm.ir.module import IRModule
+from tvm.relay import backend, testing, transform
+from tvm.relay.testing import byoc
+from tvm.relay.op.annotation import compiler_begin, compiler_end
+from aot_test_utils import (
+    AOTTestModel,
+    AOT_DEFAULT_RUNNER,
+    generate_ref_data,
+    convert_to_relay,
+    compile_and_run,
+    compile_models,
+    parametrize_aot_options,
+)
+
+
+def print_mod_tree(m, indent=0):
+    print(f"{' ' * indent} - {m!r}")
+    for i in m.imported_modules:
+        print_mod_tree(i, indent + 2)
+
+
+unpacked_api = tvm.testing.parameter(True, False)
+
+
+target_kind = tvm.testing.parameter("c", "llvm")

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] manupa-arm commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   I would expect to see something here at least : https://ci.tlcpack.ai/blue/organizations/jenkins/tvm/activity/?branch=main.
   
   I think as part of that issue the flaky ones were disabled. @lhutton1 is looking into those.  I think its worth mentioning if you see more at least in that issue, that would aid in triaging these...
   


-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;

Review comment:
       Remove

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;

Review comment:
       Remove

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;
+      if (o->IsInstance<IntImmNode>()) {
+        int64_t i = Downcast<Integer>(o);
+        Visit(nullptr, &i);
+        continue;
+      }
+
+      if (o->IsInstance<StringObj>()) {
+        std::string s = Downcast<String>(o);
+        Visit(nullptr, &s);
+        continue;
+      }
+
+      runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+      std::stringstream i_str;
+      i_str << i;
+      address_.push_back(i_str.str());
+      Visit(nullptr, &metadata);
+      address_.pop_back();
+      //      ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+    }
+    is_first_item_ = old_is_first_item;
+  }
+
+  void Visit(const char* key, ObjectRef* value) final {
+    const runtime::metadata::MetadataArrayNode* arr =
+        value->as<runtime::metadata::MetadataArrayNode>();
+    std::cout << "Is array? " << arr << std::endl;

Review comment:
       Remove

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;

Review comment:
       Remove

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;
+      if (o->IsInstance<IntImmNode>()) {
+        int64_t i = Downcast<Integer>(o);
+        Visit(nullptr, &i);
+        continue;
+      }
+
+      if (o->IsInstance<StringObj>()) {
+        std::string s = Downcast<String>(o);
+        Visit(nullptr, &s);
+        continue;
+      }
+
+      runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+      std::stringstream i_str;
+      i_str << i;
+      address_.push_back(i_str.str());
+      Visit(nullptr, &metadata);
+      address_.pop_back();
+      //      ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+    }
+    is_first_item_ = old_is_first_item;
+  }
+
+  void Visit(const char* key, ObjectRef* value) final {
+    const runtime::metadata::MetadataArrayNode* arr =
+        value->as<runtime::metadata::MetadataArrayNode>();
+    std::cout << "Is array? " << arr << std::endl;
+    if (arr != nullptr) {
+      WriteComma();
+      if (key != nullptr) {
+        address_.push_back(key);
+      }
+      code_ << address_from_parts(address_);
+      if (key != nullptr) {
+        address_.pop_back();
+      }
+      return;
+    }
+
+    runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+    if (key != nullptr) {  // NOTE: outermost call passes nullptr key
+      address_.push_back(key);
+    }
+    ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+    if (key != nullptr) {  // NOTE: outermost call passes nullptr key
+      address_.pop_back();
+    }
+  }
+
+  void CodegenMetadata(::tvm::runtime::metadata::Metadata metadata) {
+    decl_ << "#include <inttypes.h>" << std::endl
+          << "#include <tvm/runtime/metadata.h>" << std::endl
+          << "#include <tvm/runtime/c_runtime_api.h>" << std::endl;
+    std::vector<MetadataQueuer::QueueItem> queue;
+    MetadataQueuer queuer{&queue};
+    queuer.Visit(kGlobalSymbol, &metadata);
+
+    for (MetadataQueuer::QueueItem item : queue) {
+      auto struct_name = std::get<0>(item);
+      auto obj = std::get<1>(item);
+      auto arr = obj.as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "codegen: " << struct_name;

Review comment:
       Remove




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

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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -857,11 +905,30 @@ class AOTExecutorCodegen : public MixedModeVisitor {
     ICHECK(target_host_.defined()) << "require a target_host to be given for AOT codegen";
     VLOG(1) << "target host: " << target_host_->ToDebugString();
 
+    Runtime runtime_config = mod->GetAttr<Runtime>(tvm::attr::kRuntime).value();
     Executor executor_config = mod->GetAttr<Executor>(tvm::attr::kExecutor).value();
     String interface_api = executor_config->GetAttr<String>("interface-api").value_or("packed");
     Integer workspace_byte_alignment =
         executor_config->GetAttr<Integer>("workspace-byte-alignment").value_or(16);
     use_unpacked_api_ = executor_config->GetAttr<Bool>("unpacked-api").value_or(Bool(false));
+    use_call_cpacked_ = !use_unpacked_api_;
+
+    // Validate choice of use_unpacked_api_ and use_call_cpacked_
+    if (runtime_config->name == kTvmRuntimeCrt) {
+      CHECK(interface_api == "packed" || static_cast<bool>(use_unpacked_api_) == true)
+          << "Either need interface_api == \"packed\" (got: " << interface_api
+          << ") or unpacked-api == true (got: " << use_unpacked_api_
+          << ") when targeting c runtime";
+    } else if (runtime_config->name == kTvmRuntimeCpp) {
+      CHECK(static_cast<bool>(use_unpacked_api_) == false &&

Review comment:
       Should use `ICHECK`, also at L918

##########
File path: src/runtime/aot_executor/aot_executor.h
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.h
+ */
+#ifndef TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+#define TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+
+#include <tvm/runtime/metadata.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+
+#include <string>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+class TVM_DLL AotExecutor : public ModuleNode {
+ public:
+  /*!
+   * \brief Implements member function lookup for this Module for the frontend.
+   * \param name The name of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The corresponding member function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override;
+
+  /*!
+   * \return The type key of the executor.
+   */
+  const char* type_key() const final { return "AotExecutor"; }
+
+  void Run();
+
+  /*!
+   * \brief Initialize the AOT executor with metadata, runtime::Module, and device.
+   * \param module The module containing the compiled functions for the host
+   *  processor.
+   * \param devs The device of the host and devices where graph nodes will be

Review comment:
       There is no "graph".
   
   And only the first element of `devs` is ever used in `aot_executor.cc`.

##########
File path: src/target/source/source_module.cc
##########
@@ -539,6 +801,32 @@ runtime::Module CreateCSourceCrtMetadataModule(const Array<runtime::Module>& mod
   return std::move(csrc_metadata_module);
 }
 
+runtime::Module CreateCSourceCppMetadataModule(runtime::metadata::Metadata metadata) {
+  MetadataSerializer serializer;
+  serializer.CodegenMetadata(metadata);
+  std::stringstream lookup_func;
+  lookup_func << "#ifdef __cplusplus\n"
+              << "extern \"C\"\n"
+              << "#endif\n";
+
+  lookup_func << "TVM_DLL int32_t get_c_metadata(TVMValue* arg_values, int* arg_tcodes, int "
+                 "num_args, TVMValue* ret_values, int* ret_tcodes, void* resource_handle) {"
+              << std::endl;
+  lookup_func << "    ret_values[0].v_handle = (void*) &" << MetadataSerializer::kGlobalSymbol
+              << ";" << std::endl;
+  lookup_func << "    ret_tcodes[0] = kTVMOpaqueHandle;" << std::endl;
+  lookup_func << "    return 0;" << std::endl;
+  lookup_func << "};" << std::endl;
+
+  auto mod = MetadataModuleCreate(metadata);
+  std::vector<String> func_names{"get_c_metadata"};
+  // definer.GetOutput() +

Review comment:
       Remove?

##########
File path: src/target/source/codegen_source_base.h
##########
@@ -157,6 +158,19 @@ runtime::Module CSourceModuleCreate(const String& code, const String& fmt,
                                     const Array<String>& func_names,
                                     const Array<String>& const_vars = {});
 
+/*!
+ * \brief Wrap the submodules in a metadata module.
+ * \param params The variable to constant mapping that is collected by the host
+ *        module.
+ * \param target_module The main TIR-lowered internal runtime module
+ * \param modules All the external modules that needs to be imported inside the metadata module(s).
+ * \param target The target that all the modules are compiled for

Review comment:
       `metadata` not documented

##########
File path: src/tir/transforms/legalize_packed_calls.cc
##########
@@ -75,6 +75,12 @@ class PackedCallLegalizer : public StmtExprMutator {
             new_stmts.push_back(tir::Evaluate(
                 tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
                                {sid_array, 0, tir::builtin::kArrData, call->args[i]})));
+            new_stmts.push_back(tir::Evaluate(
+                tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
+                               {sid_array, 0, tir::builtin::kArrDeviceType, kDLCPU})));
+            new_stmts.push_back(tir::Evaluate(
+                tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
+                               {sid_array, 0, tir::builtin::kArrDeviceId, 0})));

Review comment:
       What are these changes for?

##########
File path: src/target/source/source_module.cc
##########
@@ -539,6 +792,32 @@ runtime::Module CreateCSourceCrtMetadataModule(const Array<runtime::Module>& mod
   return std::move(csrc_metadata_module);
 }
 
+runtime::Module CreateCSourceCppMetadataModule(runtime::metadata::Metadata metadata) {
+  MetadataSerializer serializer;
+  serializer.CodegenMetadata(metadata);
+  std::stringstream lookup_func;
+  lookup_func << "#ifdef __cplusplus\n"
+              << "extern \"C\"\n"
+              << "#endif\n";
+
+  lookup_func << "TVM_DLL int32_t get_c_metadata(TVMValue* arg_values, int* arg_tcodes, int "
+                 "num_args, TVMValue* ret_values, int* ret_tcodes, void* resource_handle) {"
+              << std::endl;
+  lookup_func << "    ret_values[0].v_handle = (void*) &" << MetadataSerializer::kGlobalSymbol
+              << ";" << std::endl;
+  lookup_func << "    ret_tcodes[0] = kTVMOpaqueHandle;" << std::endl;
+  lookup_func << "    return 0;" << std::endl;
+  lookup_func << "};" << std::endl;
+
+  auto mod = MetadataModuleCreate(metadata);
+  std::vector<String> func_names{"get_c_metadata"};

Review comment:
       Rather than hard code `get_c_metadata` everywhere, we should introduce a new symbol at https://github.com/apache/tvm/blob/d8f639a84fd9a11f03a949eb751a18018a9c1033/include/tvm/runtime/module.h#L219 (similar to `tvm_module_main`)

##########
File path: src/runtime/metadata.cc
##########
@@ -52,5 +62,65 @@ TensorInfo::TensorInfo(const struct ::TVMTensorInfo* data)
 TVM_REGISTER_OBJECT_TYPE(TensorInfoNode);
 
 }  // namespace metadata
+
+class MetadataModuleNode : public ::tvm::runtime::ModuleNode {
+ public:
+  explicit MetadataModuleNode(runtime::metadata::Metadata metadata) {
+    // CHECK((metadata.defined() && code.size() > 0) || (!metadata.defined() && code.size() == 0))
+    //   << "metadata and code must both be either defined (when passed from compiler) or undefined
+    //   "
+    //   << "(when passed from runtime)";
+    metadata_ = metadata;
+    //    code_ = code;
+  }
+
+  const char* type_key() const { return "metadata_module"; }
+
+  static Module LoadFromBinary() {
+    return Module(make_object<MetadataModuleNode>(runtime::metadata::Metadata()));
+  }
+
+  void SaveToBinary(dmlc::Stream* stream) final {}
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_metadata") {
+      return PackedFunc([this, sptr_to_self](TVMArgs args, TVMRetValue* rv) {
+        if (!metadata_.defined()) {
+          TVMFunctionHandle f_handle;
+          int32_t ret_code = TVMBackendGetFuncFromEnv(this, "get_c_metadata", &f_handle);
+          CHECK_EQ(ret_code, 0) << "Unable to locate get_c_metadata PackedFunc";
+
+          TVMValue ret_value;
+          int ret_type_code;
+          ret_code = TVMFuncCall(f_handle, nullptr, nullptr, 0, &ret_value, &ret_type_code);
+          CHECK_EQ(ret_code, 0) << "Invoking get_c_metadata: TVMFuncCall returned " << ret_code;
+
+          CHECK_EQ(ret_type_code, kTVMOpaqueHandle)
+              << "Expected kOpaqueHandle returned; got " << ret_type_code;
+          CHECK(ret_value.v_handle != nullptr) << "get_c_metadata returned nullptr";
+

Review comment:
       `ICHECK` or `ICHECK_EQ` in this function

##########
File path: src/target/metadata.h
##########
@@ -86,19 +97,22 @@ class VisitableMetadataNode : public ::tvm::runtime::metadata::MetadataNode {
 class InMemoryMetadataNode : public ::tvm::target::metadata::VisitableMetadataNode {
  public:
   InMemoryMetadataNode()
-      : InMemoryMetadataNode(0 /* version */, {} /* inputs */, {} /* outputs */,
+      : InMemoryMetadataNode(0 /* version */, {} /* inputs */, {} /* outputs */, {} /* pools */,
                              "" /* mod_name */) {}
   InMemoryMetadataNode(int64_t version,
                        const ::std::vector<::tvm::runtime::metadata::TensorInfo>& inputs,
                        const ::std::vector<::tvm::runtime::metadata::TensorInfo>& outputs,
+                       const ::std::vector<::tvm::runtime::metadata::TensorInfo>& pools,
                        const ::tvm::runtime::String mod_name)
       : VisitableMetadataNode{&storage_},
         inputs_{new struct TVMTensorInfo[inputs.size()]()},
         inputs_objs_{inputs},
         outputs_{new struct TVMTensorInfo[outputs.size()]()},
         outputs_objs_{outputs},
+        pools_{new struct TVMTensorInfo[pools.size()]()},

Review comment:
       can use `make_unique` here (as @kparzysz-quic recommended in other places)

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;
+      if (o->IsInstance<IntImmNode>()) {
+        int64_t i = Downcast<Integer>(o);
+        Visit(nullptr, &i);
+        continue;
+      }
+
+      if (o->IsInstance<StringObj>()) {
+        std::string s = Downcast<String>(o);
+        Visit(nullptr, &s);
+        continue;
+      }
+
+      runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+      std::stringstream i_str;
+      i_str << i;
+      address_.push_back(i_str.str());
+      Visit(nullptr, &metadata);
+      address_.pop_back();
+      //      ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);

Review comment:
       Remove?

##########
File path: src/runtime/aot_executor/aot_executor.h
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.h
+ */
+#ifndef TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+#define TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+
+#include <tvm/runtime/metadata.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+
+#include <string>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+class TVM_DLL AotExecutor : public ModuleNode {
+ public:
+  /*!
+   * \brief Implements member function lookup for this Module for the frontend.
+   * \param name The name of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The corresponding member function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override;
+
+  /*!
+   * \return The type key of the executor.
+   */
+  const char* type_key() const final { return "AotExecutor"; }
+
+  void Run();
+
+  /*!
+   * \brief Initialize the AOT executor with metadata, runtime::Module, and device.
+   * \param module The module containing the compiled functions for the host
+   *  processor.
+   * \param devs The device of the host and devices where graph nodes will be
+   *  executed on.
+   * \param lookup_linked_param_func If given, a PackedFunc invoked to lookup linked parameters

Review comment:
       ` lookup_linked_param_func` not given in the param list

##########
File path: src/runtime/metadata.cc
##########
@@ -52,5 +62,65 @@ TensorInfo::TensorInfo(const struct ::TVMTensorInfo* data)
 TVM_REGISTER_OBJECT_TYPE(TensorInfoNode);
 
 }  // namespace metadata
+
+class MetadataModuleNode : public ::tvm::runtime::ModuleNode {
+ public:
+  explicit MetadataModuleNode(runtime::metadata::Metadata metadata) {
+    // CHECK((metadata.defined() && code.size() > 0) || (!metadata.defined() && code.size() == 0))
+    //   << "metadata and code must both be either defined (when passed from compiler) or undefined
+    //   "
+    //   << "(when passed from runtime)";
+    metadata_ = metadata;
+    //    code_ = code;

Review comment:
       Remove




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

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

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



[GitHub] [tvm] areusch commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   @manupa-arm @kparzysz-quic please take a look when you have a minute, CI is green now.


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

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

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



[GitHub] [tvm] areusch commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;
+      if (o->IsInstance<IntImmNode>()) {
+        int64_t i = Downcast<Integer>(o);
+        Visit(nullptr, &i);
+        continue;
+      }
+
+      if (o->IsInstance<StringObj>()) {
+        std::string s = Downcast<String>(o);
+        Visit(nullptr, &s);
+        continue;
+      }
+
+      runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+      std::stringstream i_str;
+      i_str << i;
+      address_.push_back(i_str.str());
+      Visit(nullptr, &metadata);
+      address_.pop_back();
+      //      ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+    }
+    is_first_item_ = old_is_first_item;
+  }
+
+  void Visit(const char* key, ObjectRef* value) final {
+    const runtime::metadata::MetadataArrayNode* arr =
+        value->as<runtime::metadata::MetadataArrayNode>();
+    std::cout << "Is array? " << arr << std::endl;

Review comment:
       done

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;

Review comment:
       done

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;
+      if (o->IsInstance<IntImmNode>()) {
+        int64_t i = Downcast<Integer>(o);
+        Visit(nullptr, &i);
+        continue;
+      }
+
+      if (o->IsInstance<StringObj>()) {
+        std::string s = Downcast<String>(o);
+        Visit(nullptr, &s);
+        continue;
+      }
+
+      runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+      std::stringstream i_str;
+      i_str << i;
+      address_.push_back(i_str.str());
+      Visit(nullptr, &metadata);
+      address_.pop_back();
+      //      ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+    }
+    is_first_item_ = old_is_first_item;
+  }
+
+  void Visit(const char* key, ObjectRef* value) final {
+    const runtime::metadata::MetadataArrayNode* arr =
+        value->as<runtime::metadata::MetadataArrayNode>();
+    std::cout << "Is array? " << arr << std::endl;
+    if (arr != nullptr) {
+      WriteComma();
+      if (key != nullptr) {
+        address_.push_back(key);
+      }
+      code_ << address_from_parts(address_);
+      if (key != nullptr) {
+        address_.pop_back();
+      }
+      return;
+    }
+
+    runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+    if (key != nullptr) {  // NOTE: outermost call passes nullptr key
+      address_.push_back(key);
+    }
+    ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+    if (key != nullptr) {  // NOTE: outermost call passes nullptr key
+      address_.pop_back();
+    }
+  }
+
+  void CodegenMetadata(::tvm::runtime::metadata::Metadata metadata) {
+    decl_ << "#include <inttypes.h>" << std::endl
+          << "#include <tvm/runtime/metadata.h>" << std::endl
+          << "#include <tvm/runtime/c_runtime_api.h>" << std::endl;
+    std::vector<MetadataQueuer::QueueItem> queue;
+    MetadataQueuer queuer{&queue};
+    queuer.Visit(kGlobalSymbol, &metadata);
+
+    for (MetadataQueuer::QueueItem item : queue) {
+      auto struct_name = std::get<0>(item);
+      auto obj = std::get<1>(item);
+      auto arr = obj.as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "codegen: " << struct_name;

Review comment:
       done

##########
File path: src/runtime/aot_executor/aot_executor_factory.cc
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 aot_executor_factory.cc
+ * \brief Graph executor factory implementations
+ */
+
+#include "./aot_executor_factory.h"
+
+#include <tvm/runtime/container/string.h>
+#include <tvm/runtime/device_api.h>
+#include <tvm/runtime/registry.h>
+
+#include <iterator>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+AotExecutorFactory::AotExecutorFactory(

Review comment:
       it's because they are container objects which hold the constructor arguments to an executor. They also handle serializing those arguments during `export_library` and reconstructing them during `load_module`. they follow the factory pattern (e.g. a no-arg or less-args functor that constructs a given datatype), so that's why the name.

##########
File path: src/runtime/aot_executor/aot_executor_factory.cc
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 aot_executor_factory.cc
+ * \brief Graph executor factory implementations

Review comment:
       fixed

##########
File path: tests/python/relay/aot/test_crt_aot.py
##########
@@ -49,7 +50,14 @@ def test_error_c_interface_with_packed_api():
     two = relay.add(relay.const(1), relay.const(1))
     func = relay.Function([], two)
 
-    with pytest.raises(tvm.TVMError, match="Packed interface required for packed operators"):
+    with pytest.raises(
+        tvm.TVMError,
+        match=re.escape(

Review comment:
       i think the check fails when both of these conditions are false, so we just need one to assert we get the error message. the remaining single-true cases are tested in various other tests, so i think we should be ok. but lmk if you have a more specific concern.

##########
File path: src/target/source/source_module.cc
##########
@@ -539,6 +801,32 @@ runtime::Module CreateCSourceCrtMetadataModule(const Array<runtime::Module>& mod
   return std::move(csrc_metadata_module);
 }
 
+runtime::Module CreateCSourceCppMetadataModule(runtime::metadata::Metadata metadata) {
+  MetadataSerializer serializer;
+  serializer.CodegenMetadata(metadata);
+  std::stringstream lookup_func;
+  lookup_func << "#ifdef __cplusplus\n"
+              << "extern \"C\"\n"
+              << "#endif\n";
+
+  lookup_func << "TVM_DLL int32_t get_c_metadata(TVMValue* arg_values, int* arg_tcodes, int "
+                 "num_args, TVMValue* ret_values, int* ret_tcodes, void* resource_handle) {"
+              << std::endl;
+  lookup_func << "    ret_values[0].v_handle = (void*) &" << MetadataSerializer::kGlobalSymbol
+              << ";" << std::endl;
+  lookup_func << "    ret_tcodes[0] = kTVMOpaqueHandle;" << std::endl;
+  lookup_func << "    return 0;" << std::endl;
+  lookup_func << "};" << std::endl;
+
+  auto mod = MetadataModuleCreate(metadata);
+  std::vector<String> func_names{"get_c_metadata"};
+  // definer.GetOutput() +

Review comment:
       done

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;
+      if (o->IsInstance<IntImmNode>()) {
+        int64_t i = Downcast<Integer>(o);
+        Visit(nullptr, &i);
+        continue;
+      }
+
+      if (o->IsInstance<StringObj>()) {
+        std::string s = Downcast<String>(o);
+        Visit(nullptr, &s);
+        continue;
+      }
+
+      runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+      std::stringstream i_str;
+      i_str << i;
+      address_.push_back(i_str.str());
+      Visit(nullptr, &metadata);
+      address_.pop_back();
+      //      ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);

Review comment:
       done

##########
File path: src/target/source/codegen_source_base.h
##########
@@ -157,6 +158,19 @@ runtime::Module CSourceModuleCreate(const String& code, const String& fmt,
                                     const Array<String>& func_names,
                                     const Array<String>& const_vars = {});
 
+/*!
+ * \brief Wrap the submodules in a metadata module.
+ * \param params The variable to constant mapping that is collected by the host
+ *        module.
+ * \param target_module The main TIR-lowered internal runtime module
+ * \param modules All the external modules that needs to be imported inside the metadata module(s).
+ * \param target The target that all the modules are compiled for

Review comment:
       done

##########
File path: src/target/metadata_module.cc
##########
@@ -25,17 +25,145 @@
 
 #include <tvm/relay/runtime.h>
 
+#include <utility>
 #include <vector>
 
+#include "../runtime/const_loader_module.h"
 #include "../runtime/meta_data.h"
 #include "llvm/llvm_module.h"
 #include "source/source_module.h"
 
 namespace tvm {
 namespace codegen {
 
+static runtime::Module CreateCrtMetadataModule(
+    runtime::Module target_module, Target target, relay::Runtime runtime,
+    relay::backend::ExecutorCodegenMetadata metadata,
+    Array<runtime::Module> non_crt_exportable_modules,
+    Array<runtime::Module> crt_exportable_modules,
+    const std::unordered_map<std::string, runtime::NDArray>& const_var_ndarray) {
+  if (!non_crt_exportable_modules.empty()) {
+    std::string non_exportable_modules;
+    for (unsigned int i = 0; i < non_crt_exportable_modules.size(); i++) {
+      if (i > 0) {
+        non_exportable_modules += ", ";
+      }
+      auto mod = non_crt_exportable_modules[i];
+      auto pf_sym = mod.GetFunction("get_symbol");
+      if (pf_sym != nullptr) {
+        non_exportable_modules += pf_sym().operator std::string();
+      } else {
+        non_exportable_modules +=
+            std::string{"(module type_key="} + mod->type_key() + std::string{")"};
+      }
+    }
+    CHECK(false) << "These " << non_crt_exportable_modules.size()
+                 << " modules are not exportable to C-runtime: " << non_exportable_modules;
+  }
+
+  if (target->kind->name == "c") {
+    crt_exportable_modules.push_back(target_module);
+    target_module =
+        CreateCSourceCrtMetadataModule(crt_exportable_modules, target, runtime, metadata);
+  } else if (target->kind->name == "llvm") {
+#ifdef TVM_LLVM_VERSION
+    crt_exportable_modules.push_back(target_module);
+    target_module = CreateLLVMCrtMetadataModule(crt_exportable_modules, target, runtime);
+#else   // TVM_LLVM_VERSION
+    LOG(FATAL) << "TVM was not built with LLVM enabled.";
+#endif  // TVM_LLVM_VERSION
+  }
+
+  return target_module;
+}
+
+// TODO(areusch,masahi): Unify metadata representation and remove the need for this function
+static runtime::metadata::Metadata ConvertMetaData(
+    relay::backend::ExecutorCodegenMetadata metadata) {
+  std::vector<runtime::metadata::TensorInfo> inputs;
+  ICHECK(metadata.defined());
+  for (size_t i = 0; i < metadata->inputs.size(); ++i) {
+    auto v = metadata->inputs[i];
+    auto ttype = metadata->input_tensor_types[i];
+    inputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            v->name_hint, relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> outputs;
+  auto output_ttypes = metadata->output_tensor_types;
+  for (size_t i = 0; i < output_ttypes.size(); ++i) {
+    auto ttype = output_ttypes[i];
+    std::stringstream name;
+    name << "output" << i;
+    outputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            name.str(), relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> pools;
+  for (size_t i = 0; i < metadata->pools.size(); ++i) {
+    auto var = metadata->pools[i];
+    pools.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            var->name_hint,
+            std::vector<int64_t>{metadata->pool_inputs.value()[var]->allocated_size},

Review comment:
       done

##########
File path: python/tvm/runtime/executor/__init__.py
##########
@@ -0,0 +1,26 @@
+# 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.
+
+"""This module contains Python wrappers for the TVM C++ Executor implementations.
+
+NOTE: at present, only AOT Executor is contained here. The others are:
+ - GraphExecutor, in python/tvm/contrib/graph_executor.py
+ - VM Executor, in python/tvm/runtime/vm.py
+
+TODO(areusch): Consolidate these into this module.

Review comment:
       yeah that is a good point. added support here.

##########
File path: python/tvm/runtime/executor/aot_executor.py
##########
@@ -0,0 +1,182 @@
+# 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.
+"""A Python wrapper for the Module-based Model Runtime Interface for Ahead-of-Time compilation."""
+
+import numpy as np
+
+
+class AotModule(object):
+    """Wraps the AOT executor runtime.Module.
+
+    This is a thin wrapper of the underlying TVM module.
+    you can also directly call set_input, run, and get_output
+    of underlying module functions
+
+    Parameters
+    ----------
+    module : tvm.runtime.Module
+        The internal tvm module that holds the actual graph functions.
+
+    Attributes
+    ----------
+    module : tvm.runtime.Module
+        The internal tvm module that holds the actual graph functions.
+
+    Examples
+    --------
+
+    .. code-block:: python
+
+        import tvm
+        from tvm import relay
+        from tvm.contrib import graph_executor
+
+        # build the library using graph executor
+        lib = relay.build(...)
+        lib.export_library("compiled_lib.so")
+        # load it back as a runtime
+        lib: tvm.runtime.Module = tvm.runtime.load_module("compiled_lib.so")
+        # Call the library factory function for default and create
+        # a new runtime.Module, wrap with graph module.
+        gmod = graph_executor.GraphModule(lib["default"](dev))

Review comment:
       fixed, thanks!

##########
File path: src/runtime/aot_executor/aot_executor.cc
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.cc
+ */
+
+#include "aot_executor.h"
+
+#include <tvm/runtime/c_runtime_api.h>
+
+#include <memory>
+
+#include "../meta_data.h"
+
+namespace tvm {
+namespace runtime {
+
+AotExecutor::AotExecutor(tvm::runtime::Module module, const std::vector<Device>& devs)
+    : module_{module}, devices_{devs} {
+  auto fmetadata = module->GetFunction("get_metadata");
+  CHECK(fmetadata != nullptr) << "Expected a module with PackedFunc get_metadata";
+  auto ret_value = fmetadata();
+  metadata_ = ret_value.AsObjectRef<tvm::runtime::metadata::Metadata>();
+
+  for (auto input : metadata_->inputs()) {
+    // TODO(areusch): Encode device information in Metadata.
+    args_.emplace_back(NDArray::Empty(ShapeTuple(input->shape().begin(), input->shape().end()),
+                                      input->dtype(), devices_[0]));
+  }
+
+  for (auto output : metadata_->outputs()) {
+    args_.emplace_back(NDArray::Empty(ShapeTuple(output->shape().begin(), output->shape().end()),
+                                      output->dtype(), devices_[0]));
+  }
+
+  for (auto pool : metadata_->pools()) {
+    args_.emplace_back(NDArray::Empty(ShapeTuple(pool->shape().begin(), pool->shape().end()),
+                                      pool->dtype(), devices_[0]));
+  }
+}
+
+PackedFunc AotExecutor::GetFunction(const std::string& name,
+                                    const ObjectPtr<Object>& sptr_to_self) {
+  // Return member functions during query.
+  if (name == "set_input") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (String::CanConvertFrom(args[0])) {
+        int in_idx = this->GetInputIndex(args[0].operator String());
+        if (in_idx >= 0) this->SetInput(in_idx, args[1]);
+      } else {
+        this->SetInput(args[0], args[1]);
+      }
+    });
+  } else if (name == "set_input_zero_copy") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (String::CanConvertFrom(args[0])) {
+        int in_idx = this->GetInputIndex(args[0].operator String());
+        if (in_idx >= 0) this->SetInputZeroCopy(in_idx, args[1]);
+      } else {
+        this->SetInputZeroCopy(args[0], args[1]);
+      }
+    });
+  } else if (name == "set_output_zero_copy") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (String::CanConvertFrom(args[0])) {
+        int out_idx = this->GetOutputIndex(args[0].operator String());
+        if (out_idx >= 0) this->SetOutputZeroCopy(out_idx, args[1]);
+      } else {
+        this->SetOutputZeroCopy(args[0], args[1]);
+      }
+    });
+  } else if (name == "get_output") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (args.num_args == 2) {
+        this->CopyOutputTo(args[0], args[1]);
+      } else {
+        *rv = this->GetOutput(args[0]);
+      }
+    });
+  } else if (name == "get_input") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      int in_idx = 0;
+      if (String::CanConvertFrom(args[0])) {
+        in_idx = this->GetInputIndex(args[0].operator String());
+      } else {
+        in_idx = args[0];
+      }
+      if (in_idx >= 0) {
+        *rv = this->GetInput(in_idx);
+      }
+    });
+  } else if (name == "get_num_outputs") {
+    return PackedFunc(
+        [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->NumOutputs(); });
+  } else if (name == "get_num_inputs") {
+    return PackedFunc(
+        [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->NumInputs(); });
+  } else if (name == "run") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { this->Run(); });
+  } else if (name == "get_input_index") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      CHECK(String::CanConvertFrom(args[0])) << "Input key is not a string";
+      *rv = this->GetInputIndex(args[0].operator String());
+    });
+  } else {
+    return PackedFunc();
+  }
+}
+
+void AotExecutor::Run() {
+  auto pf = module_.GetFunction(
+      get_name_mangled(metadata_->mod_name(), ::tvm::runtime::symbol::tvm_module_main),
+      true /* query_imports */);
+  ICHECK(pf != nullptr) << "Module entrypoint is not defined";
+
+  const int num_args = args_.size();
+  ::std::unique_ptr<TVMValue> call_values{new TVMValue[num_args]};
+  ::std::unique_ptr<int> call_type_codes{new int[num_args]};

Review comment:
       fixed, thanks

##########
File path: include/tvm/runtime/metadata.h
##########
@@ -75,6 +75,13 @@ struct TVMMetadata {
   const struct TVMTensorInfo* outputs;
   /*! \brief Number of elements in `outputs` array. */
   int64_t num_outputs;
+  /*! \brief Memory Pools needed by the AOT run_model function.

Review comment:
       changed to just "AOT main" since the function should probably eventually be renamed based on `mod_name`.

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -857,11 +905,30 @@ class AOTExecutorCodegen : public MixedModeVisitor {
     ICHECK(target_host_.defined()) << "require a target_host to be given for AOT codegen";
     VLOG(1) << "target host: " << target_host_->ToDebugString();
 
+    Runtime runtime_config = mod->GetAttr<Runtime>(tvm::attr::kRuntime).value();
     Executor executor_config = mod->GetAttr<Executor>(tvm::attr::kExecutor).value();
     String interface_api = executor_config->GetAttr<String>("interface-api").value_or("packed");
     Integer workspace_byte_alignment =
         executor_config->GetAttr<Integer>("workspace-byte-alignment").value_or(16);
     use_unpacked_api_ = executor_config->GetAttr<Bool>("unpacked-api").value_or(Bool(false));
+    use_call_cpacked_ = !use_unpacked_api_;
+
+    // Validate choice of use_unpacked_api_ and use_call_cpacked_
+    if (runtime_config->name == kTvmRuntimeCrt) {
+      CHECK(interface_api == "packed" || static_cast<bool>(use_unpacked_api_) == true)

Review comment:
       the CRT CHECK is thoroughly tested by existing code (ran into a lot of those tests in making CI happy). added tests for the C++ one.

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -317,6 +327,16 @@ class AOTExecutorCodegen : public MixedModeVisitor {
     }
   }
 
+  void PushArgs(const Expr& expr, std::vector<tir::Var> sids, Array<PrimExpr>* args) {

Review comment:
       oops, sorry kinda rough. changed to pass-by-const-ref and pointer per c++ style guide.

##########
File path: python/tvm/runtime/executor/aot_executor.py
##########
@@ -0,0 +1,182 @@
+# 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.
+"""A Python wrapper for the Module-based Model Runtime Interface for Ahead-of-Time compilation."""
+
+import numpy as np
+
+
+class AotModule(object):

Review comment:
       i kind of find it easier to read the acronyms if we use CapWords, this also follows https://www.python.org/dev/peps/pep-0008/#class-names which is linked from numpydoc

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;
+          if (o.as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+            std::stringstream ss;
+            ss << i;
+            address_parts_.push_back(ss.str());
+            runtime::metadata::MetadataBase metadata = Downcast<runtime::metadata::MetadataBase>(o);
+            ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+            address_parts_.pop_back();
+          }
+        }
+      } else {
+        ReflectionVTable::Global()->VisitAttrs(metadata.operator->(), this);
+      }
+
+      queue_->push_back(std::make_tuple(address_from_parts(address_parts_),
+                                        Downcast<runtime::metadata::MetadataBase>(*value)));
+    }
+    address_parts_.pop_back();
+  }
+
+ private:
+  std::vector<QueueItem>* queue_;
+  std::vector<std::string> address_parts_;
+};
+
+class MetadataSerializer : public AttrVisitor {
+ public:
+  static constexpr const char* kGlobalSymbol = "kTvmgenMetadata";
+  using MetadataTypeIndex = ::tvm::runtime::metadata::MetadataTypeIndex;
+
+  MetadataSerializer() : is_first_item_{true} {}
+
+  void WriteComma() {
+    if (is_first_item_) {
+      is_first_item_ = false;
+    } else {
+      code_ << ", " << std::endl;
+    }
+  }
+
+  void WriteKey(const char* key) {
+    if (key != nullptr) {
+      code_ << " /* " << key << "*/";
+    }
+  }
+
+  void Visit(const char* key, double* value) final {
+    WriteComma();
+    code_.setf(std::ios::hex | std::ios::showbase | std::ios::fixed | std::ios::scientific,
+               std::ios::basefield | std::ios::showbase | std::ios::floatfield);
+    code_ << *value;
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, int64_t* value) final {
+    WriteComma();
+    code_ << *value << "L";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, uint64_t* value) final {
+    WriteComma();
+    code_ << *value << "UL";
+    WriteKey(key);
+  }
+  void Visit(const char* key, int* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, bool* value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, std::string* value) final {
+    WriteComma();
+    code_ << "\"" << *value << "\"";
+    WriteKey(key);
+  }
+  void Visit(const char* key, void** value) final {
+    WriteComma();
+    code_ << *value;
+    WriteKey(key);
+  }
+  void Visit(const char* key, DataType* value) final {
+    WriteComma();
+    code_ << "{" << value->code() << ", " << value->bits() << ", " << value->lanes() << "}";
+    WriteKey(key);
+  }
+
+  void Visit(const char* key, runtime::NDArray* value) final {
+    // TODO(areusch): probably we could consolidate --link-params here, tho...
+    ICHECK(false) << "do not support serializing NDArray as metadata";
+  }
+
+  void VisitArray(const runtime::metadata::MetadataArrayNode* array) {
+    std::cout << "visit array " << array << ": " << array->type_index << " " << array->struct_name
+              << "," << array->array.size() << std::endl;
+    auto old_is_first_item = is_first_item_;
+    is_first_item_ = true;
+    for (unsigned int i = 0; i < array->array.size(); ++i) {  // ObjectRef o : *(array->array)) {
+      ObjectRef o = array->array[i];
+      std::cout << "visiting array element " << i << ": " << o->type_index() << " ("
+                << o.operator->() << ")" << std::endl;

Review comment:
       done

##########
File path: src/target/source/source_module.cc
##########
@@ -518,6 +522,264 @@ class CSourceCrtMetadataModuleNode : public runtime::ModuleNode {
   }
 };
 
+static std::string address_from_parts(const std::vector<std::string>& parts) {
+  std::stringstream ss;
+  for (unsigned int i = 0; i < parts.size(); ++i) {
+    if (i > 0) {
+      ss << "_";
+    }
+    ss << parts[i];
+  }
+  return ss.str();
+}
+
+class MetadataQueuer : public AttrVisitor {
+ public:
+  using QueueItem = std::tuple<std::string, runtime::metadata::MetadataBase>;
+  explicit MetadataQueuer(std::vector<QueueItem>* queue) : queue_{queue} {}
+
+  void Visit(const char* key, double* value) final {}
+  void Visit(const char* key, int64_t* value) final {}
+  void Visit(const char* key, uint64_t* value) final {}
+  void Visit(const char* key, int* value) final {}
+  void Visit(const char* key, bool* value) final {}
+  void Visit(const char* key, std::string* value) final {}
+  void Visit(const char* key, DataType* value) final {}
+  void Visit(const char* key, runtime::NDArray* value) final {}
+  void Visit(const char* key, void** value) final {}
+
+  void Visit(const char* key, ObjectRef* value) final {
+    address_parts_.push_back(key);
+    if (value->as<runtime::metadata::MetadataBaseNode>() != nullptr) {
+      auto metadata = Downcast<runtime::metadata::MetadataBase>(*value);
+      const runtime::metadata::MetadataArrayNode* arr =
+          value->as<runtime::metadata::MetadataArrayNode>();
+      std::cout << "Is array? " << arr << std::endl;
+      if (arr != nullptr) {
+        for (unsigned int i = 0; i < arr->array.size(); i++) {
+          ObjectRef o = arr->array[i];
+          std::cout << "queue-visiting array element " << i << ": " << o->type_index() << " ("
+                    << o.operator->() << ")" << std::endl;

Review comment:
       done

##########
File path: src/runtime/metadata.cc
##########
@@ -52,5 +62,65 @@ TensorInfo::TensorInfo(const struct ::TVMTensorInfo* data)
 TVM_REGISTER_OBJECT_TYPE(TensorInfoNode);
 
 }  // namespace metadata
+
+class MetadataModuleNode : public ::tvm::runtime::ModuleNode {
+ public:
+  explicit MetadataModuleNode(runtime::metadata::Metadata metadata) {
+    // CHECK((metadata.defined() && code.size() > 0) || (!metadata.defined() && code.size() == 0))
+    //   << "metadata and code must both be either defined (when passed from compiler) or undefined
+    //   "
+    //   << "(when passed from runtime)";
+    metadata_ = metadata;
+    //    code_ = code;

Review comment:
       done

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -857,11 +905,30 @@ class AOTExecutorCodegen : public MixedModeVisitor {
     ICHECK(target_host_.defined()) << "require a target_host to be given for AOT codegen";
     VLOG(1) << "target host: " << target_host_->ToDebugString();
 
+    Runtime runtime_config = mod->GetAttr<Runtime>(tvm::attr::kRuntime).value();
     Executor executor_config = mod->GetAttr<Executor>(tvm::attr::kExecutor).value();
     String interface_api = executor_config->GetAttr<String>("interface-api").value_or("packed");
     Integer workspace_byte_alignment =
         executor_config->GetAttr<Integer>("workspace-byte-alignment").value_or(16);
     use_unpacked_api_ = executor_config->GetAttr<Bool>("unpacked-api").value_or(Bool(false));
+    use_call_cpacked_ = !use_unpacked_api_;
+
+    // Validate choice of use_unpacked_api_ and use_call_cpacked_
+    if (runtime_config->name == kTvmRuntimeCrt) {
+      CHECK(interface_api == "packed" || static_cast<bool>(use_unpacked_api_) == true)
+          << "Either need interface_api == \"packed\" (got: " << interface_api
+          << ") or unpacked-api == true (got: " << use_unpacked_api_
+          << ") when targeting c runtime";
+    } else if (runtime_config->name == kTvmRuntimeCpp) {
+      CHECK(static_cast<bool>(use_unpacked_api_) == false &&

Review comment:
       fixed

##########
File path: src/runtime/aot_executor/aot_executor.h
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.h
+ */
+#ifndef TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+#define TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+
+#include <tvm/runtime/metadata.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+
+#include <string>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+class TVM_DLL AotExecutor : public ModuleNode {
+ public:
+  /*!
+   * \brief Implements member function lookup for this Module for the frontend.
+   * \param name The name of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The corresponding member function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override;
+
+  /*!
+   * \return The type key of the executor.
+   */
+  const char* type_key() const final { return "AotExecutor"; }
+
+  void Run();
+
+  /*!
+   * \brief Initialize the AOT executor with metadata, runtime::Module, and device.
+   * \param module The module containing the compiled functions for the host
+   *  processor.
+   * \param devs The device of the host and devices where graph nodes will be

Review comment:
       reworded the comment, added ICHECK, and tested.

##########
File path: src/target/metadata.h
##########
@@ -86,19 +97,22 @@ class VisitableMetadataNode : public ::tvm::runtime::metadata::MetadataNode {
 class InMemoryMetadataNode : public ::tvm::target::metadata::VisitableMetadataNode {
  public:
   InMemoryMetadataNode()
-      : InMemoryMetadataNode(0 /* version */, {} /* inputs */, {} /* outputs */,
+      : InMemoryMetadataNode(0 /* version */, {} /* inputs */, {} /* outputs */, {} /* pools */,
                              "" /* mod_name */) {}
   InMemoryMetadataNode(int64_t version,
                        const ::std::vector<::tvm::runtime::metadata::TensorInfo>& inputs,
                        const ::std::vector<::tvm::runtime::metadata::TensorInfo>& outputs,
+                       const ::std::vector<::tvm::runtime::metadata::TensorInfo>& pools,
                        const ::tvm::runtime::String mod_name)
       : VisitableMetadataNode{&storage_},
         inputs_{new struct TVMTensorInfo[inputs.size()]()},
         inputs_objs_{inputs},
         outputs_{new struct TVMTensorInfo[outputs.size()]()},
         outputs_objs_{outputs},
+        pools_{new struct TVMTensorInfo[pools.size()]()},

Review comment:
       i tried this but it complained about it couldn't find the right constructor. i think maybe make_unique needs to be used with operator=, so it's actually not appropriate here.

##########
File path: src/runtime/aot_executor/aot_executor.h
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.h
+ */
+#ifndef TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+#define TVM_RUNTIME_AOT_EXECUTOR_AOT_EXECUTOR_H_
+
+#include <tvm/runtime/metadata.h>
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/packed_func.h>
+
+#include <string>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+class TVM_DLL AotExecutor : public ModuleNode {
+ public:
+  /*!
+   * \brief Implements member function lookup for this Module for the frontend.
+   * \param name The name of the function.
+   * \param sptr_to_self The pointer to the module node.
+   * \return The corresponding member function.
+   */
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override;
+
+  /*!
+   * \return The type key of the executor.
+   */
+  const char* type_key() const final { return "AotExecutor"; }
+
+  void Run();
+
+  /*!
+   * \brief Initialize the AOT executor with metadata, runtime::Module, and device.
+   * \param module The module containing the compiled functions for the host
+   *  processor.
+   * \param devs The device of the host and devices where graph nodes will be
+   *  executed on.
+   * \param lookup_linked_param_func If given, a PackedFunc invoked to lookup linked parameters

Review comment:
       fixed

##########
File path: src/tir/transforms/legalize_packed_calls.cc
##########
@@ -75,6 +75,12 @@ class PackedCallLegalizer : public StmtExprMutator {
             new_stmts.push_back(tir::Evaluate(
                 tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
                                {sid_array, 0, tir::builtin::kArrData, call->args[i]})));
+            new_stmts.push_back(tir::Evaluate(
+                tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
+                               {sid_array, 0, tir::builtin::kArrDeviceType, kDLCPU})));
+            new_stmts.push_back(tir::Evaluate(
+                tvm::tir::Call(DataType::Handle(), tvm::tir::builtin::tvm_struct_set(),
+                               {sid_array, 0, tir::builtin::kArrDeviceId, 0})));

Review comment:
       these changes fixed the flaky stack bug where device allocations were using bad threadlocalstorage for workspace. the fix was to explicitly initialize the device_id and device_type fields of the created DLTensor. there are other fields we need to initialize as well, but which we don't use in the c-codgen generated code, at least not extensively.

##########
File path: src/target/source/source_module.cc
##########
@@ -539,6 +792,32 @@ runtime::Module CreateCSourceCrtMetadataModule(const Array<runtime::Module>& mod
   return std::move(csrc_metadata_module);
 }
 
+runtime::Module CreateCSourceCppMetadataModule(runtime::metadata::Metadata metadata) {
+  MetadataSerializer serializer;
+  serializer.CodegenMetadata(metadata);
+  std::stringstream lookup_func;
+  lookup_func << "#ifdef __cplusplus\n"
+              << "extern \"C\"\n"
+              << "#endif\n";
+
+  lookup_func << "TVM_DLL int32_t get_c_metadata(TVMValue* arg_values, int* arg_tcodes, int "
+                 "num_args, TVMValue* ret_values, int* ret_tcodes, void* resource_handle) {"
+              << std::endl;
+  lookup_func << "    ret_values[0].v_handle = (void*) &" << MetadataSerializer::kGlobalSymbol
+              << ";" << std::endl;
+  lookup_func << "    ret_tcodes[0] = kTVMOpaqueHandle;" << std::endl;
+  lookup_func << "    return 0;" << std::endl;
+  lookup_func << "};" << std::endl;
+
+  auto mod = MetadataModuleCreate(metadata);
+  std::vector<String> func_names{"get_c_metadata"};

Review comment:
       done

##########
File path: src/runtime/metadata.cc
##########
@@ -52,5 +62,65 @@ TensorInfo::TensorInfo(const struct ::TVMTensorInfo* data)
 TVM_REGISTER_OBJECT_TYPE(TensorInfoNode);
 
 }  // namespace metadata
+
+class MetadataModuleNode : public ::tvm::runtime::ModuleNode {
+ public:
+  explicit MetadataModuleNode(runtime::metadata::Metadata metadata) {
+    // CHECK((metadata.defined() && code.size() > 0) || (!metadata.defined() && code.size() == 0))
+    //   << "metadata and code must both be either defined (when passed from compiler) or undefined
+    //   "
+    //   << "(when passed from runtime)";
+    metadata_ = metadata;
+    //    code_ = code;
+  }
+
+  const char* type_key() const { return "metadata_module"; }
+
+  static Module LoadFromBinary() {
+    return Module(make_object<MetadataModuleNode>(runtime::metadata::Metadata()));
+  }
+
+  void SaveToBinary(dmlc::Stream* stream) final {}
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) {
+    if (name == "get_metadata") {
+      return PackedFunc([this, sptr_to_self](TVMArgs args, TVMRetValue* rv) {
+        if (!metadata_.defined()) {
+          TVMFunctionHandle f_handle;
+          int32_t ret_code = TVMBackendGetFuncFromEnv(this, "get_c_metadata", &f_handle);
+          CHECK_EQ(ret_code, 0) << "Unable to locate get_c_metadata PackedFunc";
+
+          TVMValue ret_value;
+          int ret_type_code;
+          ret_code = TVMFuncCall(f_handle, nullptr, nullptr, 0, &ret_value, &ret_type_code);
+          CHECK_EQ(ret_code, 0) << "Invoking get_c_metadata: TVMFuncCall returned " << ret_code;
+
+          CHECK_EQ(ret_type_code, kTVMOpaqueHandle)
+              << "Expected kOpaqueHandle returned; got " << ret_type_code;
+          CHECK(ret_value.v_handle != nullptr) << "get_c_metadata returned nullptr";
+

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] kparzysz-quic commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

Posted by GitBox <gi...@apache.org>.
kparzysz-quic commented on a change in pull request #10283:
URL: https://github.com/apache/tvm/pull/10283#discussion_r816210034



##########
File path: src/runtime/aot_executor/aot_executor.cc
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.
+ */
+
+/*!
+ * \brief Defines an implementation of Module-based Model Runtime Interface that works with
+ *        Ahead-of-Time compilation.
+ * \file aot_executor.cc
+ */
+
+#include "aot_executor.h"
+
+#include <tvm/runtime/c_runtime_api.h>
+
+#include <memory>
+
+#include "../meta_data.h"
+
+namespace tvm {
+namespace runtime {
+
+AotExecutor::AotExecutor(tvm::runtime::Module module, const std::vector<Device>& devs)
+    : module_{module}, devices_{devs} {
+  auto fmetadata = module->GetFunction("get_metadata");
+  CHECK(fmetadata != nullptr) << "Expected a module with PackedFunc get_metadata";
+  auto ret_value = fmetadata();
+  metadata_ = ret_value.AsObjectRef<tvm::runtime::metadata::Metadata>();
+
+  for (auto input : metadata_->inputs()) {
+    // TODO(areusch): Encode device information in Metadata.
+    args_.emplace_back(NDArray::Empty(ShapeTuple(input->shape().begin(), input->shape().end()),
+                                      input->dtype(), devices_[0]));
+  }
+
+  for (auto output : metadata_->outputs()) {
+    args_.emplace_back(NDArray::Empty(ShapeTuple(output->shape().begin(), output->shape().end()),
+                                      output->dtype(), devices_[0]));
+  }
+
+  for (auto pool : metadata_->pools()) {
+    args_.emplace_back(NDArray::Empty(ShapeTuple(pool->shape().begin(), pool->shape().end()),
+                                      pool->dtype(), devices_[0]));
+  }
+}
+
+PackedFunc AotExecutor::GetFunction(const std::string& name,
+                                    const ObjectPtr<Object>& sptr_to_self) {
+  // Return member functions during query.
+  if (name == "set_input") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (String::CanConvertFrom(args[0])) {
+        int in_idx = this->GetInputIndex(args[0].operator String());
+        if (in_idx >= 0) this->SetInput(in_idx, args[1]);
+      } else {
+        this->SetInput(args[0], args[1]);
+      }
+    });
+  } else if (name == "set_input_zero_copy") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (String::CanConvertFrom(args[0])) {
+        int in_idx = this->GetInputIndex(args[0].operator String());
+        if (in_idx >= 0) this->SetInputZeroCopy(in_idx, args[1]);
+      } else {
+        this->SetInputZeroCopy(args[0], args[1]);
+      }
+    });
+  } else if (name == "set_output_zero_copy") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (String::CanConvertFrom(args[0])) {
+        int out_idx = this->GetOutputIndex(args[0].operator String());
+        if (out_idx >= 0) this->SetOutputZeroCopy(out_idx, args[1]);
+      } else {
+        this->SetOutputZeroCopy(args[0], args[1]);
+      }
+    });
+  } else if (name == "get_output") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      if (args.num_args == 2) {
+        this->CopyOutputTo(args[0], args[1]);
+      } else {
+        *rv = this->GetOutput(args[0]);
+      }
+    });
+  } else if (name == "get_input") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      int in_idx = 0;
+      if (String::CanConvertFrom(args[0])) {
+        in_idx = this->GetInputIndex(args[0].operator String());
+      } else {
+        in_idx = args[0];
+      }
+      if (in_idx >= 0) {
+        *rv = this->GetInput(in_idx);
+      }
+    });
+  } else if (name == "get_num_outputs") {
+    return PackedFunc(
+        [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->NumOutputs(); });
+  } else if (name == "get_num_inputs") {
+    return PackedFunc(
+        [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->NumInputs(); });
+  } else if (name == "run") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { this->Run(); });
+  } else if (name == "get_input_index") {
+    return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+      CHECK(String::CanConvertFrom(args[0])) << "Input key is not a string";
+      *rv = this->GetInputIndex(args[0].operator String());
+    });
+  } else {
+    return PackedFunc();
+  }
+}
+
+void AotExecutor::Run() {
+  auto pf = module_.GetFunction(
+      get_name_mangled(metadata_->mod_name(), ::tvm::runtime::symbol::tvm_module_main),
+      true /* query_imports */);
+  ICHECK(pf != nullptr) << "Module entrypoint is not defined";
+
+  const int num_args = args_.size();
+  ::std::unique_ptr<TVMValue> call_values{new TVMValue[num_args]};
+  ::std::unique_ptr<int> call_type_codes{new int[num_args]};

Review comment:
       These two lines should be
   ```
   auto call_values = std::make_unique<TVMValue[]>(num_args);
   auto call_type_codes = std::make_unique<int[]>(num_args);
   ```

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -177,6 +179,12 @@ class AOTOnDemandAllocator : public transform::DeviceAwareExprVisitor {
       for (auto sid : sinfo->storage_ids) {
         return_ids_.push_back(sid);
       }
+      return_ttypes_.clear();
+      auto ttypes = FlattenTupleType(e->checked_type());
+      return_ttypes_.reserve(ttypes.size());
+      for (auto ttype : ttypes) {
+        return_ttypes_.push_back(ttype);
+      }

Review comment:
       All of this is equivalent to `return_ttypes_ = FlattenTupleType(e->checked_type());`.




-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: python/tvm/runtime/executor/__init__.py
##########
@@ -0,0 +1,26 @@
+# 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.
+
+"""This module contains Python wrappers for the TVM C++ Executor implementations.
+
+NOTE: at present, only AOT Executor is contained here. The others are:
+ - GraphExecutor, in python/tvm/contrib/graph_executor.py
+ - VM Executor, in python/tvm/runtime/vm.py
+
+TODO(areusch): Consolidate these into this module.

Review comment:
       I just realized that, we have two notions of `executor`. One is the runtime one above, the other is 
   https://github.com/apache/tvm/blob/55849e651e3dff6cffeb7fd14aa89699e9575d10/python/tvm/relay/build_module.py#L647
   which is used a lot in the test cases.
   
   Do we intend to support `create_executor(kind="aot", ...)`, given that we can now run things via the cpp runtime??




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

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

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



[GitHub] [tvm] areusch commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: tests/python/relay/aot/test_cpp_aot.py
##########
@@ -0,0 +1,118 @@
+# 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.
+
+
+import sys
+import textwrap
+
+import numpy as np
+import pytest
+
+import tvm
+from tvm import relay, TVMError
+from tvm.ir.module import IRModule
+from tvm.relay import backend, testing, transform
+from tvm.relay.testing import byoc
+from tvm.relay.op.annotation import compiler_begin, compiler_end
+from aot_test_utils import (
+    AOTTestModel,
+    AOT_DEFAULT_RUNNER,
+    generate_ref_data,
+    convert_to_relay,
+    compile_and_run,
+    compile_models,
+    parametrize_aot_options,
+)
+
+
+def print_mod_tree(m, indent=0):
+    print(f"{' ' * indent} - {m!r}")
+    for i in m.imported_modules:
+        print_mod_tree(i, indent + 2)
+
+
+unpacked_api = tvm.testing.parameter(True, False)
+
+
+target_kind = tvm.testing.parameter("c", "llvm")

Review comment:
       yeah sorry this test is still crufty and needs some polish. will do before merge.




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

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 commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   @areusch Could the ethos-u failure be a flaky one? We had a lot of them recently.


-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/target/metadata_module.cc
##########
@@ -25,17 +25,145 @@
 
 #include <tvm/relay/runtime.h>
 
+#include <utility>
 #include <vector>
 
+#include "../runtime/const_loader_module.h"
 #include "../runtime/meta_data.h"
 #include "llvm/llvm_module.h"
 #include "source/source_module.h"
 
 namespace tvm {
 namespace codegen {
 
+static runtime::Module CreateCrtMetadataModule(
+    runtime::Module target_module, Target target, relay::Runtime runtime,
+    relay::backend::ExecutorCodegenMetadata metadata,
+    Array<runtime::Module> non_crt_exportable_modules,
+    Array<runtime::Module> crt_exportable_modules,
+    const std::unordered_map<std::string, runtime::NDArray>& const_var_ndarray) {
+  if (!non_crt_exportable_modules.empty()) {
+    std::string non_exportable_modules;
+    for (unsigned int i = 0; i < non_crt_exportable_modules.size(); i++) {
+      if (i > 0) {
+        non_exportable_modules += ", ";
+      }
+      auto mod = non_crt_exportable_modules[i];
+      auto pf_sym = mod.GetFunction("get_symbol");
+      if (pf_sym != nullptr) {
+        non_exportable_modules += pf_sym().operator std::string();
+      } else {
+        non_exportable_modules +=
+            std::string{"(module type_key="} + mod->type_key() + std::string{")"};
+      }
+    }
+    CHECK(false) << "These " << non_crt_exportable_modules.size()
+                 << " modules are not exportable to C-runtime: " << non_exportable_modules;
+  }
+
+  if (target->kind->name == "c") {
+    crt_exportable_modules.push_back(target_module);
+    target_module =
+        CreateCSourceCrtMetadataModule(crt_exportable_modules, target, runtime, metadata);
+  } else if (target->kind->name == "llvm") {
+#ifdef TVM_LLVM_VERSION
+    crt_exportable_modules.push_back(target_module);
+    target_module = CreateLLVMCrtMetadataModule(crt_exportable_modules, target, runtime);
+#else   // TVM_LLVM_VERSION
+    LOG(FATAL) << "TVM was not built with LLVM enabled.";
+#endif  // TVM_LLVM_VERSION
+  }
+
+  return target_module;
+}
+
+// TODO(areusch,masahi): Unify metadata representation and remove the need for this function
+static runtime::metadata::Metadata ConvertMetaData(
+    relay::backend::ExecutorCodegenMetadata metadata) {
+  std::vector<runtime::metadata::TensorInfo> inputs;
+  ICHECK(metadata.defined());
+  for (size_t i = 0; i < metadata->inputs.size(); ++i) {
+    auto v = metadata->inputs[i];
+    auto ttype = metadata->input_tensor_types[i];
+    inputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            v->name_hint, relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> outputs;
+  auto output_ttypes = metadata->output_tensor_types;
+  for (size_t i = 0; i < output_ttypes.size(); ++i) {
+    auto ttype = output_ttypes[i];
+    std::stringstream name;
+    name << "output" << i;
+    outputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            name.str(), relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> pools;
+  for (size_t i = 0; i < metadata->pools.size(); ++i) {
+    auto var = metadata->pools[i];
+    pools.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            var->name_hint,
+            std::vector<int64_t>{metadata->pool_inputs.value()[var]->allocated_size},

Review comment:
       Is it 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] masahi commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: python/tvm/runtime/executor/__init__.py
##########
@@ -0,0 +1,26 @@
+# 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.
+
+"""This module contains Python wrappers for the TVM C++ Executor implementations.
+
+NOTE: at present, only AOT Executor is contained here. The others are:
+ - GraphExecutor, in python/tvm/contrib/graph_executor.py
+ - VM Executor, in python/tvm/runtime/vm.py
+
+TODO(areusch): Consolidate these into this module.

Review comment:
       I just realized that, we have two notions of `executor`. One is the runtime one above, the other is 
   https://github.com/apache/tvm/blob/55849e651e3dff6cffeb7fd14aa89699e9575d10/python/tvm/relay/build_module.py#L647
   
   Do we intend to support `create_executor(kind="aot", ...)`?




-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: tests/python/relay/aot/test_cpp_aot.py
##########
@@ -0,0 +1,118 @@
+# 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.
+
+
+import sys
+import textwrap
+
+import numpy as np
+import pytest
+
+import tvm
+from tvm import relay, TVMError
+from tvm.ir.module import IRModule
+from tvm.relay import backend, testing, transform
+from tvm.relay.testing import byoc
+from tvm.relay.op.annotation import compiler_begin, compiler_end
+from aot_test_utils import (
+    AOTTestModel,
+    AOT_DEFAULT_RUNNER,
+    generate_ref_data,
+    convert_to_relay,
+    compile_and_run,
+    compile_models,
+    parametrize_aot_options,
+)
+
+
+def print_mod_tree(m, indent=0):
+    print(f"{' ' * indent} - {m!r}")
+    for i in m.imported_modules:
+        print_mod_tree(i, indent + 2)
+
+
+unpacked_api = tvm.testing.parameter(True, False)
+
+
+target_kind = tvm.testing.parameter("c", "llvm")

Review comment:
       need to remove llvm for now




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

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

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



[GitHub] [tvm] areusch commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   @Mousius @manupa-arm embarrassingly i can't quite figure out how to reproduce the Ethos-U test failures locally:
   ```
   $ docker/bash.sh -it ci_qemu bash -c 'cd build && cmake -GNinja .. && ninja && cd .. && tests/scripts/task_ci_s
   etup.sh && . tests/scripts/setup-pytest-env.sh  && python3  -mpytest tests/python/contrib/test_ethosu -k "test_tflite_squeeze[ifm_shape1-3-ethos-
   u55-256]"'
   
   ...
   ==================================================================== ERRORS =====================================================================
   _________________________________________________________ ERROR collecting test session _________________________________________________________
   /usr/lib/python3.6/importlib/__init__.py:126: in import_module
       return _bootstrap._gcd_import(name[level:], package, level)
   <frozen importlib._bootstrap>:994: in _gcd_import
       ???
   <frozen importlib._bootstrap>:971: in _find_and_load
       ???
   <frozen importlib._bootstrap>:955: in _find_and_load_unlocked
       ???
   <frozen importlib._bootstrap>:665: in _load_unlocked
       ???
   /opt/tvm-venv/lib/python3.6/site-packages/_pytest/assertion/rewrite.py:170: in exec_module
       exec(co, module.__dict__)
   tests/python/contrib/test_ethosu/cascader/conftest.py:49: in <module>
       from ..infra import (
   tests/python/contrib/test_ethosu/infra.py:44: in <module>
       from tests.python.relay.aot.aot_test_utils import (
   E   ModuleNotFoundError: No module named 'tests.python'
   ```
   
   I can't find where we add tests as a valid import path. I have a vague recollection we discussed this but can't find where it was. do you know? hoping the CI passes if i fixed the unittests.


-- 
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] manupa-arm commented on pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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


   The above sounds like a import issue with tests modules.
   
   @masahi is it the same issue you were seeing? If so it shouldn't not be flaky rather it should fail always.
   
   I can have a look tommorow but unfortunately haven't encountered it personally.


-- 
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 commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: include/tvm/runtime/metadata.h
##########
@@ -75,6 +75,13 @@ struct TVMMetadata {
   const struct TVMTensorInfo* outputs;
   /*! \brief Number of elements in `outputs` array. */
   int64_t num_outputs;
+  /*! \brief Memory Pools needed by the AOT run_model function.

Review comment:
       You are not using `run_model` anymore, are you? In your previous branch indeed I'm seeing `tvmgen_default_run_model` generated, but after rebase it is replaced with `tvmgen_default___tvm_main__`.




-- 
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] areusch commented on a change in pull request #10283: [runtime] AOTExecutor implementation and c target code-generator

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



##########
File path: src/target/metadata_module.cc
##########
@@ -25,17 +25,145 @@
 
 #include <tvm/relay/runtime.h>
 
+#include <utility>
 #include <vector>
 
+#include "../runtime/const_loader_module.h"
 #include "../runtime/meta_data.h"
 #include "llvm/llvm_module.h"
 #include "source/source_module.h"
 
 namespace tvm {
 namespace codegen {
 
+static runtime::Module CreateCrtMetadataModule(
+    runtime::Module target_module, Target target, relay::Runtime runtime,
+    relay::backend::ExecutorCodegenMetadata metadata,
+    Array<runtime::Module> non_crt_exportable_modules,
+    Array<runtime::Module> crt_exportable_modules,
+    const std::unordered_map<std::string, runtime::NDArray>& const_var_ndarray) {
+  if (!non_crt_exportable_modules.empty()) {
+    std::string non_exportable_modules;
+    for (unsigned int i = 0; i < non_crt_exportable_modules.size(); i++) {
+      if (i > 0) {
+        non_exportable_modules += ", ";
+      }
+      auto mod = non_crt_exportable_modules[i];
+      auto pf_sym = mod.GetFunction("get_symbol");
+      if (pf_sym != nullptr) {
+        non_exportable_modules += pf_sym().operator std::string();
+      } else {
+        non_exportable_modules +=
+            std::string{"(module type_key="} + mod->type_key() + std::string{")"};
+      }
+    }
+    CHECK(false) << "These " << non_crt_exportable_modules.size()
+                 << " modules are not exportable to C-runtime: " << non_exportable_modules;
+  }
+
+  if (target->kind->name == "c") {
+    crt_exportable_modules.push_back(target_module);
+    target_module =
+        CreateCSourceCrtMetadataModule(crt_exportable_modules, target, runtime, metadata);
+  } else if (target->kind->name == "llvm") {
+#ifdef TVM_LLVM_VERSION
+    crt_exportable_modules.push_back(target_module);
+    target_module = CreateLLVMCrtMetadataModule(crt_exportable_modules, target, runtime);
+#else   // TVM_LLVM_VERSION
+    LOG(FATAL) << "TVM was not built with LLVM enabled.";
+#endif  // TVM_LLVM_VERSION
+  }
+
+  return target_module;
+}
+
+// TODO(areusch,masahi): Unify metadata representation and remove the need for this function
+static runtime::metadata::Metadata ConvertMetaData(
+    relay::backend::ExecutorCodegenMetadata metadata) {
+  std::vector<runtime::metadata::TensorInfo> inputs;
+  ICHECK(metadata.defined());
+  for (size_t i = 0; i < metadata->inputs.size(); ++i) {
+    auto v = metadata->inputs[i];
+    auto ttype = metadata->input_tensor_types[i];
+    inputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            v->name_hint, relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> outputs;
+  auto output_ttypes = metadata->output_tensor_types;
+  for (size_t i = 0; i < output_ttypes.size(); ++i) {
+    auto ttype = output_ttypes[i];
+    std::stringstream name;
+    name << "output" << i;
+    outputs.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            name.str(), relay::backend::ShapeToJSON(ttype->shape), ttype->dtype)));
+  }
+
+  std::vector<runtime::metadata::TensorInfo> pools;
+  for (size_t i = 0; i < metadata->pools.size(); ++i) {
+    auto var = metadata->pools[i];
+    pools.push_back(
+        runtime::metadata::TensorInfo(make_object<target::metadata::InMemoryTensorInfoNode>(
+            var->name_hint,
+            std::vector<int64_t>{metadata->pool_inputs.value()[var]->allocated_size},

Review comment:
       it is done earlier in the function, but it was in the wrong spot and i've reorganized the checks.

##########
File path: src/relay/backend/aot_executor_codegen.cc
##########
@@ -177,6 +179,12 @@ class AOTOnDemandAllocator : public transform::DeviceAwareExprVisitor {
       for (auto sid : sinfo->storage_ids) {
         return_ids_.push_back(sid);
       }
+      return_ttypes_.clear();
+      auto ttypes = FlattenTupleType(e->checked_type());
+      return_ttypes_.reserve(ttypes.size());
+      for (auto ttype : ttypes) {
+        return_ttypes_.push_back(ttype);
+      }

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