You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/05/18 19:59:22 UTC

[GitHub] [tvm] areusch opened a new pull request #8072: Add "operator" style to Model Library Format

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


   This PR introduces a `style` key to the Model Library Format metadata with two initial values: 
    - `full-model`
    - `operator`
   
   Implementations that use Model Library Format will now need to check `style` before reading other files in the archive. `full-model` indicates the previously-used format.
   
   `operator` is introduced to allow exporting TVM libraries that contain only operator functions with no model-level information (e.g. executor configuration, model-wide memory planning, etc). The goal of `operator` style is to allow exporting fragments of models (e.g. individual TVM operators) for use with the TVM RPC Server. After the Project API refactor lands, TVM auto-tuning will produce MLF in `operator` style, and those MLF archives will be given to project generators with the ultimate goal of flashing and timing those operators on-device.
   
    MLF archives with `operator` style contain:
    - `codegen` directory, organized as in `full-model`
    - `metadata.json` of the same format as `full-model` with different values.
    - `src/tir-<device_type>.txt`, containing pretty-printed TIR
   
   Notably, the `memory` key in `metadata.json` contains shape information for each operator function parameter. The shape information has names correlated with those used in the TIR sources in `src/tir-*.txt`.
   
   @leandron @manupa-arm @giuseros @Mousius @gromero @mehrdadh @stoa


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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       I was actually referring to other objects with member functions.
   E.g. : 
   https://github.com/manupa-arm/incubator-tvm/blob/master/src/relay/analysis/call_graph.cc 
   https://github.com/manupa-arm/incubator-tvm/blob/master/python/tvm/relay/analysis/call_graph.py
   
   There are similiar structure in AutoScheduler as well. I always thought this was better than extending runtime.Modules and using packed functions. What do you think?




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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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


   @giuseros @manupa-arm please take a look, i think I've addressed your comments or replied on thread


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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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






-- 
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 #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       we need `GetFunction` to expose an object to Python with member functions, and i think the member function style provides a bit more structure to the API than just e.g. placing it all in a tuple. I don't think we can use Node since it doesn't provide the `sptr_to_self`




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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       If we are going with this approach, I feel we should limit this to just to the lookup ladder of functions.
   i.e., its better to implement the lambda functions as separate functions
   
   Moreover, since this is main interface to runtime.Module, I think we should provide documentation of the functions and arguments.




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

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



[GitHub] [tvm] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)
+        with open(src_dir / f"tir-{target_device_type}.txt", "w") as f:
+            f.write(printer["print"](ir_mod))
+
+        for v in ir_mod.get_global_vars():
+            map_entry = []
+            for p, b in ir_mod[v.name_hint].buffer_map.items():
+                shape = _eval_shape(p.name, b.shape)
+                buffer_size_bytes = _shape_to_size(shape, str(b.dtype))
+                # NOTE: cannot tell what is an input or output at this point.
+                map_entry.append(
+                    {
+                        "size_bytes": buffer_size_bytes,
+                        "shape": [int(x) for x in b.shape],
+                        "dtype": b.dtype,
+                        "input_binding": printer["get_var_name"](p),
+                    }
+                )
+            memory_map[v.name_hint] = map_entry
+
+    return memory_map
+
+
+def _export_operator_model_library_format(mod: build_module.OperatorModule, tempdir):
+    """Export the result of tvm.build() in Model Library Format.
+
+    Parameters
+    ----------
+    mod : runtime.Module
+        The Module returned from tvm.build().
+    args : list of Buffer or Tensor or Var, optional
+        The args supplied to tvm.build().
+    file_name : str
+        Path to the .tar archive to generate.
+    """
+    targets = {}
+    for target in mod.ir_module_by_target.keys():
+        if str(target.kind) not in ("llvm", "c"):
+            raise UnsupportedInModelLibraryFormatError(
+                f"Operator has non-DSO-exportable target {target!s}, which is not yet supported in "
+                "Model Library Format"
+            )
+
+        targets[int(_nd.device(str(target)).device_type)] = target
+
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    memory_map = _write_tir_and_build_operator_memory_map(src_dir, targets, mod.ir_module_by_target)
+
+    metadata = {
+        "version": _GENERATED_VERSION,
+        "model_name": mod.name,
+        "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
+        "memory": memory_map,
+        "target": {k: str(v) for k, v in targets.items()},
+        "runtimes": [],
+        "style": "operator",
+    }
+    with open(tempdir / "metadata.json", "w") as metadata_f:
+        json.dump(metadata, metadata_f)
+
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod, codegen_dir)
+
+
+ExportableModule = typing.Union[

Review comment:
       i guess this is a bit specific to Model Library Format--you can build shared libraries from things we don't know how to export into MLF. happy to change the name, or we can revisit this when we promote MLF to a top-level TVM export format.




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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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


   friendly ping! this one is blocking Project API work


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

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



[GitHub] [tvm] leandron merged pull request #8072: Add "operator" style to Model Library Format

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


   


-- 
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 #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)

Review comment:
       yeah good point. in `src/printer`, we have a few entry points:
    - `src/printer/tvmscript_printer.cc` defines `script.AsTVMScript`
    - `src/printer/text_printer.cc` defines `ir.PrettyPrint` and `ir.AsText`
   
   so i guess the folder doesn't provide any namespace grouping right now, even though printer implementations are consolidated there. i'm okay moving to `micro.ModelLibraryFormatPrinter` or `ir.ModelLibraryFormatPrinter`, if that's what you're suggesting. `tir` seemed like a fit since that's how we are using it now, though it should work with any IRModule.
   
   could you let me know which namespace you're suggesting to move to?




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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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


   @manupa-arm please let me know if there's anything else--i believe your comments are all forward-looking, but want to understand if there are specific changes needed here to 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] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       hmm. i think that this way is a more codified dispatch mechanism (e.g. string table lookup) than that used by Module (`GetFunction`  typically implemented with a series of if statements and closures), but it requires additional duplication of each member function in e.g. Python. I think a proper interface-oriented FFI would just wrap all of this stuff automatically.
   
   my preference with interface FFI functions in TVM is to use `Module` in general, but add some automation to the build to auto-generate `GetFunction` and avoid closures in that function. It is a bit weird, I'll admit, since it re-uses a mechanism meant for the runtime. But, I think it's the only generic member-function lookup we have right now, and there are some non-runtime use cases.

##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)

Review comment:
       in this case, we need the member function to retrieve the mapping--this is why i used Module. as for the namespace, i don't have a strong opinion, but the only `micro` directory we have in `src` is `src/runtime`, and this is clearly not a runtime component. so we'd need to create `src/micro`, is all. i'm not opposed to that, but was following convention for `Printer` in keeping `ModelLibraryFormatPrinter` underneath `src/printer`, is all.




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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       Yes, I think there is precedence for both approaches. Lets not block this based on this :). 
   Maybe its better to converge to a single policy when exposing member functions across the FFI. 
   cc : @tqchen @jroesch .

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       Yes, I think there is precedence for both approaches. Lets not block this based on this :). 
   Maybe its better to converge to a single policy when exposing member functions across the FFI. 
   cc : @tqchen @jroesch .
   
   Though, if the intention is of this class to be used in python, I'd still prefer to have a documented interface in python -- it seems much easier to follow than the indirections via GetFunction.

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       If we are going with this approach, I feel we should limit this to just the lookup table of functions.
   i.e., its better to implement the lambda functions as separate functions
   
   Moreover, since this is main interface to runtime.Module, I think we should provide documentation of the functions and arguments.

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       If we are going with this approach, I feel we should limit this to just to the lookup ladder of functions.
   i.e., its better to implement the lambda functions as separate functions
   
   Moreover, since this is main interface to runtime.Module, I think we should provide documentation of the functions and arguments.

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "print") {
+      return TypedPackedFunc<std::string(ObjectRef)>(
+          [sptr_to_self, this](ObjectRef node) { return Print(node); });
+    } else if (name == "get_var_name") {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        ICHECK_EQ(args.size(), 1) << "usage: get_var_name(Var v)";
+
+        std::string var_name;
+        if (text_printer_.GetVarName(args[0], &var_name)) {
+          *rv = var_name;
+        }
+      });
+    } else {
+      return PackedFunc();
+    }
+  }
+
+ private:
+  TextPrinter text_printer_;
+};
+
+TVM_REGISTER_GLOBAL("tir.ModelLibraryFormatPrinter")

Review comment:
       This seems fine as it works for all TIR -- I just got reminded that we moved MLF to micro for a different reason.

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       If we are going with this approach, I feel we should limit this to just to the lookup ladder of functions.
   i.e., its better to implement the lambda functions as separate functions
   
   Moreover, since this is main interface to runtime.Module, I think we should provide documentation of the functions and arguments -- maybe once the functions are seperated out, it could be the documentation of those functions.




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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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


   oops, think someone already took MLF version 3. added a patch to rev to v4.
   
   @giuseros please take a look and explicitly approve if you're ok with this PR!


-- 
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 #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "print") {
+      return TypedPackedFunc<std::string(ObjectRef)>(
+          [sptr_to_self, this](ObjectRef node) { return Print(node); });
+    } else if (name == "get_var_name") {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        ICHECK_EQ(args.size(), 1) << "usage: get_var_name(Var v)";
+
+        std::string var_name;
+        if (text_printer_.GetVarName(args[0], &var_name)) {
+          *rv = var_name;
+        }

Review comment:
       i guess the thinking is that the caller can decide what to do. since the function will return None, it should be fairly straightforward. we could raise an exception. are you thinking that users may not check the return value?

##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path

Review comment:
       done




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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       Yes, I think there is precedence for both approaches. Lets not block this based on this :). 
   Maybe its better to converge to a single policy when exposing member functions across the FFI. 
   cc : @tqchen @jroesch .
   
   Though, if the intention is of this class to be used in python, I'd still prefer to have a documented interface in python -- it seems much easier to follow than the indirections via GetFunction.




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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       If we are going with this approach, I feel we should limit this to just the lookup table of functions.
   i.e., its better to implement the lambda functions as separate functions
   
   Moreover, since this is main interface to runtime.Module, I think we should provide documentation of the functions and arguments.




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

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



[GitHub] [tvm] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       i agree with that--however, we do need the lambda function to capture `sptr_to_self` (this mimics the Python descriptor `get()` implementation). i moved the body into a separate function to align this class for a future world where we implemented the auto-generated interface.
   
   cc @jroesch who has a prototype of the auto-generator




-- 
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 #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       Yes, I think there is precedence for both approaches. Lets not block this based on this :). 
   Maybe its better to converge to a single policy when exposing member functions across the FFI. 
   cc : @tqchen @jroesch .




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

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



[GitHub] [tvm] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "print") {
+      return TypedPackedFunc<std::string(ObjectRef)>(
+          [sptr_to_self, this](ObjectRef node) { return Print(node); });
+    } else if (name == "get_var_name") {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        ICHECK_EQ(args.size(), 1) << "usage: get_var_name(Var v)";
+
+        std::string var_name;
+        if (text_printer_.GetVarName(args[0], &var_name)) {
+          *rv = var_name;
+        }

Review comment:
       i guess the thinking is that the caller can decide what to do. since the function will return None, it should be fairly straightforward. we could raise an exception. are you thinking that users may not check the return value?

##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path

Review comment:
       done




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

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



[GitHub] [tvm] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)
+        with open(src_dir / f"tir-{target_device_type}.txt", "w") as f:
+            f.write(printer["print"](ir_mod))

Review comment:
       it's mostly an analogy to adding the relay.txt into the archive--to provide TVM source code for the generated code. though I see your point that TIR is quite close to the generated code.




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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "print") {
+      return TypedPackedFunc<std::string(ObjectRef)>(
+          [sptr_to_self, this](ObjectRef node) { return Print(node); });
+    } else if (name == "get_var_name") {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        ICHECK_EQ(args.size(), 1) << "usage: get_var_name(Var v)";
+
+        std::string var_name;
+        if (text_printer_.GetVarName(args[0], &var_name)) {
+          *rv = var_name;
+        }
+      });
+    } else {
+      return PackedFunc();
+    }
+  }
+
+ private:
+  TextPrinter text_printer_;
+};
+
+TVM_REGISTER_GLOBAL("tir.ModelLibraryFormatPrinter")

Review comment:
       This seems fine as it works for all TIR -- I just got reminded that we moved MLF to micro for a different reason.




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

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



[GitHub] [tvm] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       i agree with that--however, we do need the lambda function to capture `sptr_to_self` (this mimics the Python descriptor `get()` implementation). i moved the body into a separate function to align this class for a future world where we implemented the auto-generated interface.
   
   cc @jroesch who has a prototype of the auto-generator




-- 
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 #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)
+        with open(src_dir / f"tir-{target_device_type}.txt", "w") as f:
+            f.write(printer["print"](ir_mod))
+
+        for v in ir_mod.get_global_vars():
+            map_entry = []
+            for p, b in ir_mod[v.name_hint].buffer_map.items():
+                shape = _eval_shape(p.name, b.shape)
+                buffer_size_bytes = _shape_to_size(shape, str(b.dtype))
+                # NOTE: cannot tell what is an input or output at this point.
+                map_entry.append(
+                    {
+                        "size_bytes": buffer_size_bytes,
+                        "shape": [int(x) for x in b.shape],
+                        "dtype": b.dtype,
+                        "input_binding": printer["get_var_name"](p),
+                    }
+                )
+            memory_map[v.name_hint] = map_entry
+
+    return memory_map
+
+
+def _export_operator_model_library_format(mod: build_module.OperatorModule, tempdir):
+    """Export the result of tvm.build() in Model Library Format.
+
+    Parameters
+    ----------
+    mod : runtime.Module
+        The Module returned from tvm.build().
+    args : list of Buffer or Tensor or Var, optional
+        The args supplied to tvm.build().
+    file_name : str
+        Path to the .tar archive to generate.
+    """
+    targets = {}
+    for target in mod.ir_module_by_target.keys():
+        if str(target.kind) not in ("llvm", "c"):
+            raise UnsupportedInModelLibraryFormatError(
+                f"Operator has non-DSO-exportable target {target!s}, which is not yet supported in "
+                "Model Library Format"
+            )
+
+        targets[int(_nd.device(str(target)).device_type)] = target
+
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    memory_map = _write_tir_and_build_operator_memory_map(src_dir, targets, mod.ir_module_by_target)
+
+    metadata = {
+        "version": _GENERATED_VERSION,
+        "model_name": mod.name,
+        "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
+        "memory": memory_map,
+        "target": {k: str(v) for k, v in targets.items()},
+        "runtimes": [],
+        "style": "operator",
+    }
+    with open(tempdir / "metadata.json", "w") as metadata_f:
+        json.dump(metadata, metadata_f)
+
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod, codegen_dir)
+
+
+ExportableModule = typing.Union[

Review comment:
       Ack, sounds good for now then.




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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       Why are we extending runtime.Module ? 
   Any reason why we cant use Objects and Nodes to expose this to python?

##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)

Review comment:
       I feel we can have this neatly hidden under _ffi_api.py and move the c++ implementations related to ModelLibraryFormatPrinter to a matching model_library_format.cc.
   
   Why do we think ModelLibraryFormatPrinter belongs to the namespace of tir?

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "print") {
+      return TypedPackedFunc<std::string(ObjectRef)>(
+          [sptr_to_self, this](ObjectRef node) { return Print(node); });
+    } else if (name == "get_var_name") {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        ICHECK_EQ(args.size(), 1) << "usage: get_var_name(Var v)";
+
+        std::string var_name;
+        if (text_printer_.GetVarName(args[0], &var_name)) {
+          *rv = var_name;
+        }
+      });
+    } else {
+      return PackedFunc();
+    }
+  }
+
+ private:
+  TextPrinter text_printer_;
+};
+
+TVM_REGISTER_GLOBAL("tir.ModelLibraryFormatPrinter")

Review comment:
       See my comment above about the source code arrangement

##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)
+        with open(src_dir / f"tir-{target_device_type}.txt", "w") as f:
+            f.write(printer["print"](ir_mod))
+
+        for v in ir_mod.get_global_vars():
+            map_entry = []
+            for p, b in ir_mod[v.name_hint].buffer_map.items():
+                shape = _eval_shape(p.name, b.shape)
+                buffer_size_bytes = _shape_to_size(shape, str(b.dtype))
+                # NOTE: cannot tell what is an input or output at this point.
+                map_entry.append(
+                    {
+                        "size_bytes": buffer_size_bytes,
+                        "shape": [int(x) for x in b.shape],
+                        "dtype": b.dtype,
+                        "input_binding": printer["get_var_name"](p),
+                    }
+                )
+            memory_map[v.name_hint] = map_entry
+
+    return memory_map
+
+
+def _export_operator_model_library_format(mod: build_module.OperatorModule, tempdir):
+    """Export the result of tvm.build() in Model Library Format.
+
+    Parameters
+    ----------
+    mod : runtime.Module
+        The Module returned from tvm.build().
+    args : list of Buffer or Tensor or Var, optional
+        The args supplied to tvm.build().
+    file_name : str
+        Path to the .tar archive to generate.
+    """
+    targets = {}
+    for target in mod.ir_module_by_target.keys():
+        if str(target.kind) not in ("llvm", "c"):
+            raise UnsupportedInModelLibraryFormatError(
+                f"Operator has non-DSO-exportable target {target!s}, which is not yet supported in "
+                "Model Library Format"
+            )
+
+        targets[int(_nd.device(str(target)).device_type)] = target
+
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    memory_map = _write_tir_and_build_operator_memory_map(src_dir, targets, mod.ir_module_by_target)
+
+    metadata = {
+        "version": _GENERATED_VERSION,
+        "model_name": mod.name,
+        "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
+        "memory": memory_map,
+        "target": {k: str(v) for k, v in targets.items()},
+        "runtimes": [],
+        "style": "operator",
+    }
+    with open(tempdir / "metadata.json", "w") as metadata_f:
+        json.dump(metadata, metadata_f)
+
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod, codegen_dir)
+
+
+ExportableModule = typing.Union[

Review comment:
       Not sure whether the model_library_format.py is the right place to hold this




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

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



[GitHub] [tvm] leandron commented on pull request #8072: Add "operator" style to Model Library Format

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


   Merged now, thanks @manupa-arm @giuseros @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] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)

Review comment:
       I was thinking of "micro.model_library_format.printer" being the registration and make printer a python function that binds to C++ under _ffi_api.py (similiar to how its done in CallGraph).




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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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


   @giuseros @manupa-arm please take a look, i think I've addressed your comments or replied on thread


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

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



[GitHub] [tvm] manupa-arm commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {

Review comment:
       I was actually referring to other objects with member functions.
   E.g. : 
   https://github.com/manupa-arm/incubator-tvm/blob/master/src/relay/analysis/call_graph.cc 
   https://github.com/manupa-arm/incubator-tvm/blob/master/python/tvm/relay/analysis/call_graph.py
   
   There are similiar structure in AutoScheduler as well. I always this was better than extending runtime.Modules and using packed functions. What do you think?




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

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



[GitHub] [tvm] areusch commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)

Review comment:
       yeah good point. in `src/printer`, we have a few entry points:
    - `src/printer/tvmscript_printer.cc` defines `script.AsTVMScript`
    - `src/printer/text_printer.cc` defines `ir.PrettyPrint` and `ir.AsText`
   
   so i guess the folder doesn't provide any namespace grouping right now, even though printer implementations are consolidated there. i'm okay moving to `micro.ModelLibraryFormatPrinter` or `ir.ModelLibraryFormatPrinter`, if that's what you're suggesting. `tir` seemed like a fit since that's how we are using it now, though it should work with any IRModule.
    - `src/printer/




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

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



[GitHub] [tvm] areusch commented on pull request #8072: Add "operator" style to Model Library Format

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


   @giuseros please take another look and explicitly approve if you're ok with this


-- 
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 #8072: Add "operator" style to Model Library Format

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



##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {

Review comment:
       If we are going with this approach, I feel we should limit this to just to the lookup ladder of functions.
   i.e., its better to implement the lambda functions as separate functions
   
   Moreover, since this is main interface to runtime.Module, I think we should provide documentation of the functions and arguments -- maybe once the functions are seperated out, it could be the documentation of those functions.




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

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



[GitHub] [tvm] giuseros commented on a change in pull request #8072: Add "operator" style to Model Library Format

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



##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path

Review comment:
       Shouldn't this be ExecutorFactoryModule to be compatible with AOT as well?

##########
File path: src/printer/model_library_format_printer.cc
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <tvm/runtime/module.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/tir/var.h>
+
+#include "text_printer.h"
+
+namespace tvm {
+namespace printer {
+
+class ModelLibraryFormatPrinter : public ::tvm::runtime::ModuleNode {
+ public:
+  ModelLibraryFormatPrinter(bool show_meta_data,
+                            const runtime::TypedPackedFunc<std::string(ObjectRef)>& annotate,
+                            bool show_warning)
+      : text_printer_{show_meta_data, annotate, show_warning} {}
+
+  const char* type_key() const override { return "model_library_format_printer"; }
+
+  std::string Print(const ObjectRef& node) {
+    Doc doc;
+    doc << text_printer_.PrintFinal(node);
+    return doc.str();
+  }
+
+  PackedFunc GetFunction(const std::string& name, const ObjectPtr<Object>& sptr_to_self) override {
+    if (name == "print") {
+      return TypedPackedFunc<std::string(ObjectRef)>(
+          [sptr_to_self, this](ObjectRef node) { return Print(node); });
+    } else if (name == "get_var_name") {
+      return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) {
+        ICHECK_EQ(args.size(), 1) << "usage: get_var_name(Var v)";
+
+        std::string var_name;
+        if (text_printer_.GetVarName(args[0], &var_name)) {
+          *rv = var_name;
+        }

Review comment:
       Should this ICHECK if GetVarName returns false?

##########
File path: python/tvm/micro/model_library_format.py
##########
@@ -203,60 +209,200 @@ def _build_function_memory_map(function_metadata):
     return ret
 
 
-def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name):
-    """Export the build artifact in Model Library Format.
+def _make_tar(source_dir, tar_file_path):
+    """Build a tar file from source_dir."""
+    with tarfile.open(tar_file_path, "w") as tar_f:
 
-    This function creates a .tar archive containing the build artifacts in a standardized
-    layout. It's intended to allow downstream automation to build TVM artifacts against the C
-    runtime.
+        def reset(tarinfo):
+            tarinfo.uid = tarinfo.gid = 0
+            tarinfo.uname = tarinfo.gname = "root"
+            return tarinfo
+
+        tar_f.add(str(source_dir), arcname=".", filter=reset)
+
+
+_GENERATED_VERSION = 2
+
+
+def _export_graph_model_library_format(
+    mod: executor_factory.GraphExecutorFactoryModule, tempdir: pathlib.Path
+):
+    """Export a tvm.relay.build artifact in Model Library Format.
 
     Parameters
     ----------
     mod : tvm.relay.backend.executor_factory.ExecutorFactoryModule
         The return value of tvm.relay.build, which will be exported into Model Library Format.
-    file_name : str
-        Path to the .tar archive to generate.
+    tempdir : pathlib.Path
+        Temporary directory to populate with Model Library Format contents.
     """
-    tempdir = utils.tempdir()
     is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule)
     runtime = ["aot"] if is_aot else ["graph"]
 
     metadata = {
-        "version": 2,
+        "version": _GENERATED_VERSION,
         "model_name": mod.libmod_name,
         "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"),
         "memory": _build_memory_map(mod),
         "target": {int(k): str(v) for k, v in mod.target.items()},
         "runtimes": runtime,
+        "style": "full-model",
     }
 
-    with open(tempdir.relpath("metadata.json"), "w") as json_f:
+    with open(tempdir / "metadata.json", "w") as json_f:
         json.dump(metadata, json_f, indent=2, sort_keys=True)
 
-    codegen_dir_path = tempdir.relpath("codegen")
-    os.mkdir(codegen_dir_path)
-    _populate_codegen_dir(mod.lib, codegen_dir_path)
+    codegen_dir = tempdir / "codegen"
+    codegen_dir.mkdir()
+    _populate_codegen_dir(mod.lib, codegen_dir)
 
-    parameters_dir_path = tempdir.relpath("parameters")
-    os.mkdir(parameters_dir_path)
-    param_filename = os.path.join(parameters_dir_path, f"{mod.libmod_name}.params")
+    parameters_dir = tempdir / "parameters"
+    parameters_dir.mkdir()
+    param_filename = parameters_dir / f"{mod.libmod_name}.params"
     with open(param_filename, "wb") as f:
         f.write(param_dict.save_param_dict(mod.params))
 
-    with open(tempdir.relpath("relay.txt"), "w") as f:
+    src_dir = tempdir / "src"
+    src_dir.mkdir()
+    with open(src_dir / "relay.txt", "w") as f:
         f.write(str(mod.ir_mod))
 
     if not is_aot:
-        graph_config_dir_path = tempdir.relpath(os.path.join("runtime-config", "graph"))
-        os.makedirs(graph_config_dir_path)
-        with open(os.path.join(graph_config_dir_path, "graph.json"), "w") as f:
+        graph_config_dir = tempdir / "runtime-config" / "graph"
+        graph_config_dir.mkdir(parents=True)
+        with open(graph_config_dir / "graph.json", "w") as f:
             f.write(mod.get_executor_config())
 
-    with tarfile.open(file_name, "w") as tar_f:
 
-        def reset(tarinfo):
-            tarinfo.uid = tarinfo.gid = 0
-            tarinfo.uname = tarinfo.gname = "root"
-            return tarinfo
+class NonStaticShapeError(Exception):
+    """Raised when a shape has elements other than IntImm."""
+
+
+def _shape_to_size(shape, dtype):
+    bits_per_item = int(
+        re.match(r"((float)|(int))(?P<width_bits>[0-9]+)", dtype).group("width_bits")
+    )
+    assert bits_per_item is not None, f"don't know how to compute size of type {dtype}"
+    total_bits = bits_per_item
+    for s in shape:
+        total_bits *= s
+
+    return (total_bits + 7) // 8
+
+
+def _write_tir_and_build_operator_memory_map(src_dir, targets, ir_module_by_target):
+    def _eval_shape(param_name, buffer_shape):
+        shape = []
+        for x in buffer_shape:
+            if not isinstance(x, expr.IntImm):
+                raise NonStaticShapeError(
+                    f"Parameter {param_name} has shape with non-IntImm elements: {buffer_shape}"
+                )
+            shape.append(x.value)
+        return shape
+
+    memory_map = {}
+    for target_device_type, target in targets.items():
+        ir_mod = ir_module_by_target[target]
+        printer = get_global_func("tir.ModelLibraryFormatPrinter")(False, None, False)
+        with open(src_dir / f"tir-{target_device_type}.txt", "w") as f:
+            f.write(printer["print"](ir_mod))

Review comment:
       I am not following why adding the TIR in the archive. Is this for test purposes? 




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

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