diff --git a/python/tvm/micro/model_library_format.py b/python/tvm/micro/model_library_format.py index 4fd85ea38d98..be991e22a0f8 100644 --- a/python/tvm/micro/model_library_format.py +++ b/python/tvm/micro/model_library_format.py @@ -27,6 +27,9 @@ from ..relay.backend import executor_factory from ..relay import param_dict +# This should be kept identical to runtime::symbol::tvm_module_main +MAIN_FUNC_NAME_STR = "__tvm_main__" + class UnsupportedInModelLibraryFormatError(Exception): """Raised when export_model_library_format does not support the given Module tree.""" @@ -73,8 +76,16 @@ def _populate_codegen_dir(mod, codegen_dir: str): dso_mod.save(file_name) -def _build_memory_map(graph_json): - """Build a simpler memory map from graph JSON. +def _build_memory_map(mod): + ret = dict() + if isinstance(mod, executor_factory.GraphExecutorFactoryModule): + ret["sids"] = _build_sid_map(mod.graph_json) + ret["functions"] = _build_function_memory_map(mod.function_metadata) + return ret + + +def _build_sid_map(graph_json): + """Build a simpler storage id info map from graph JSON. Parameters ---------- @@ -117,6 +128,81 @@ def _build_memory_map(graph_json): return memory_map +def _build_function_memory_map(function_metadata): + """Build a simple map that shows how much workspace is required to execute + each primitive function. The main_func describes how much memory is required + to execute the main control code. + + Parameters + ---------- + function_metadata : Map + This contains all the compiled metadata on a function basis + + Returns + ------- + dict : + This will have two entries: + 1.) A list with one entry per function describing local memory it is using. + 2.) A global memory requirement if all functions are executed sequentially + """ + device_max_workspace = dict() + main_func_metadata = function_metadata[MAIN_FUNC_NAME_STR] + num_targets = len(main_func_metadata.workspace_sizes.items()) + func_entries = [] + target_local_entries = dict() + for i in range(num_targets): + target = main_func_metadata.workspace_sizes.items()[i][0] + device_max_workspace[target] = 0 + for func_name, finfo in function_metadata.items(): + if func_name == MAIN_FUNC_NAME_STR: + continue + target_local_entries[func_name] = list() + + for func_name, finfo in function_metadata.items(): + if func_name == MAIN_FUNC_NAME_STR: + continue + assert len(finfo.constant_sizes.items()) == num_targets + assert len(finfo.io_sizes.items()) == num_targets + target = finfo.workspace_sizes.items()[i][0] + workspace_size = finfo.workspace_sizes.items()[i][1] + target_entry = { + "device": int(target.kind.device_type), + "workspace_size_bytes": int(workspace_size), + } + target_local_entries[func_name].append(target_entry) + if workspace_size > device_max_workspace[target]: + device_max_workspace[target] = workspace_size + + for func_name, target_entries_ in target_local_entries.items(): + func_entry = { + "function_name": str(func_name), + "workspace": target_entries_, + } + func_entries.append(func_entry) + + target_main_entries = list() + for i in range(num_targets): + target = main_func_metadata.workspace_sizes.items()[i][0] + main_func_local_workspace = main_func_metadata.workspace_sizes.items()[i][1] + main_func_constants = main_func_metadata.constant_sizes.items()[i][1] + main_func_io = main_func_metadata.io_sizes.items()[i][1] + target_main_entries.append( + { + "device": int(target.kind.device_type), + "workspace_size_bytes": int(device_max_workspace[target]) + + int(main_func_local_workspace), + "constants_size_bytes": int(main_func_constants), + "io_size_bytes": int(main_func_io), + } + ) + + ret = { + "operator_functions": func_entries, + "main": target_main_entries, + } + return ret + + def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, file_name): """Export the build artifact in Model Library Format. @@ -133,14 +219,13 @@ def export_model_library_format(mod: executor_factory.ExecutorFactoryModule, fil """ tempdir = utils.tempdir() is_aot = isinstance(mod, executor_factory.AOTExecutorFactoryModule) - memory_map = [] if is_aot else _build_memory_map(mod.get_executor_config()) runtime = ["aot"] if is_aot else ["graph"] metadata = { - "version": 1, + "version": 2, "model_name": mod.libmod_name, "export_datetime": datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%SZ"), - "memory": memory_map, + "memory": _build_memory_map(mod), "target": {int(k): str(v) for k, v in mod.target.items()}, "runtimes": runtime, } diff --git a/python/tvm/relay/backend/executor_factory.py b/python/tvm/relay/backend/executor_factory.py index f81d8f9f1c15..4ed76f4b6366 100644 --- a/python/tvm/relay/backend/executor_factory.py +++ b/python/tvm/relay/backend/executor_factory.py @@ -81,15 +81,18 @@ class AOTExecutorFactoryModule(ExecutorFactoryModule): The name of module params : dict of str to NDArray The parameters of module + function_metadata : Map of String to FunctionInfo + This holds a map function names to their information """ - def __init__(self, ir_mod, target, libmod, libmod_name, params): + def __init__(self, ir_mod, target, libmod, libmod_name, params, function_metadata): self.ir_mod = ir_mod self.target = target self.lib = libmod self.libmod_name = libmod_name self.params = params self.iter_cnt = 0 + self.function_metadata = function_metadata def get_params(self): return self.params @@ -118,9 +121,13 @@ class GraphExecutorFactoryModule(ExecutorFactoryModule): The name of module params : dict of str to NDArray The parameters of module + function_metadata : Map of String to FunctionInfo + This holds a map function names to their information """ - def __init__(self, ir_mod, target, graph_json_str, libmod, libmod_name, params): + def __init__( + self, ir_mod, target, graph_json_str, libmod, libmod_name, params, function_metadata + ): assert isinstance(graph_json_str, string_types) fcreate = get_global_func("tvm.graph_executor_factory.create") args = [] @@ -136,6 +143,7 @@ def __init__(self, ir_mod, target, graph_json_str, libmod, libmod_name, params): self.libmod_name = libmod_name self.params = params self.iter_cnt = 0 + self.function_metadata = function_metadata def export_library(self, file_name, fcompile=None, addons=None, **kwargs): return self.module.export_library(file_name, fcompile, addons, **kwargs) diff --git a/python/tvm/relay/build_module.py b/python/tvm/relay/build_module.py index 2d8c8207c930..e134eeeefd09 100644 --- a/python/tvm/relay/build_module.py +++ b/python/tvm/relay/build_module.py @@ -83,6 +83,7 @@ def __init__(self): self._optimize = self.mod["optimize"] self._set_params_func = self.mod["set_params"] self._get_params_func = self.mod["get_params"] + self._get_function_metadata = self.mod["get_function_metadata"] def build(self, mod, target=None, target_host=None, params=None, executor="graph"): """ @@ -200,6 +201,12 @@ def get_module(self): """Return the built module.""" return self._get_module() + def get_function_metadata(self): + """Return the compiled function metadata. + Currently, the metadata contains workspace size required by + each PrimFunc""" + return self._get_function_metadata() + def get_params(self): """Return the updated weights.""" params = self._get_params_func() @@ -325,14 +332,15 @@ def build(ir_mod, target=None, target_host=None, params=None, mod_name="default" executor_config, runtime_mod, params = bld_mod.build( mod=ir_mod, target=target, params=params, executor=executor ) + func_metadata = bld_mod.get_function_metadata() if executor == "aot": executor_factory = _executor_factory.AOTExecutorFactoryModule( - ir_mod, target, runtime_mod, mod_name, params + ir_mod, target, runtime_mod, mod_name, params, func_metadata ) elif executor == "graph": executor_factory = _executor_factory.GraphExecutorFactoryModule( - ir_mod, target, executor_config, runtime_mod, mod_name, params + ir_mod, target, executor_config, runtime_mod, mod_name, params, func_metadata ) else: assert False, "Executor " + executor + " not supported" diff --git a/src/relay/backend/aot_executor_codegen.cc b/src/relay/backend/aot_executor_codegen.cc index 1939e05e2075..ef188b9df175 100644 --- a/src/relay/backend/aot_executor_codegen.cc +++ b/src/relay/backend/aot_executor_codegen.cc @@ -25,8 +25,11 @@ #include #include #include +#include +#include #include #include +#include #include #include @@ -270,6 +273,79 @@ class AOTExecutorCodegen : public ExprVisitor { return ss.str(); } + /*! + * \brief Update the "main" control function's metadata + * + * \param func The main function that contains calls to operator tir primitive functions + */ + void UpdateMainWorkspaceSize(const tir::PrimFunc& primfunc, const relay::Function& func) { + Integer workspace_size = CalculateWorkspaceBytes(primfunc); + // Populate FunctionInfo + auto fi_node = make_object(); + // Initialize all target workspaces to zero + for (const auto& kv : targets_) { + auto tgt = kv.second; + fi_node->workspace_sizes.Set(tgt, 0); + } + fi_node->workspace_sizes.Set(target_host_, workspace_size); + fi_node->relay_primfuncs.Set(target_host_, func); + + int64_t io_size = 0; + for (const auto& input : input_vars_) { + io_size += CalculateRelayExprSizeBytes(input->checked_type()); + } + io_size += CalculateRelayExprSizeBytes(func->body->checked_type()); + fi_node->io_sizes.Set(target_host_, io_size); + + int64_t const_size = 0; + for (const auto& kv : params_by_expr_) { + const_size += CalculateRelayExprSizeBytes(kv.first->checked_type()); + } + fi_node->constant_sizes.Set(target_host_, const_size); + function_metadata_.Set(String(runtime::symbol::tvm_module_main), FunctionInfo(fi_node)); + } + + /*! + * \brief Update the function metadata for a given cached function and its relay + * primitive function. + * + * \param cfunc The cached function as provided the by the compile engine + * \param relay_func The source relay primitive function + * \param relay_target The target associated with relay primitive function + */ + void UpdateFunctionMetadata(const CachedFunc& cfunc, const Function& relay_func, + const Target& relay_target) { + auto fi_node = make_object(); + for (const auto& kv : cfunc->funcs->functions) { + auto primfunc = Downcast(kv.second); + Integer workspace_size = CalculateWorkspaceBytes(primfunc); + Target primfunc_target = relay_target; + if (primfunc->attrs->dict.count("target")) { + primfunc_target = Downcast(primfunc->attrs->dict["target"]); + } + fi_node->workspace_sizes.Set(primfunc_target, workspace_size); + // Calculating size for I/O + for (auto const& param : primfunc->params) { + auto p_shape = primfunc->buffer_map[param]->shape; + int num_of_elements = 1; + for (const auto& dim_index_expr : p_shape) { + if (dim_index_expr->IsInstance()) { + num_of_elements *= dim_index_expr.as()->value; + } else { + // If shape is dynamic, we cannot calculate workspace in compile time. + num_of_elements = 0; + } + } + int element_size = primfunc->buffer_map[param]->dtype.bytes(); + fi_node->io_sizes.Set(primfunc_target, element_size * num_of_elements); + } + fi_node->constant_sizes.Set(primfunc_target, 0); + fi_node->tir_primfuncs.Set(primfunc_target, primfunc); + fi_node->relay_primfuncs.Set(primfunc_target, relay_func); + } + function_metadata_.Set(cfunc->func_name, FunctionInfo(fi_node)); + } + void VisitExpr_(const CallNode* op) override { // Descend the call tree for (auto arg : op->args) { @@ -336,6 +412,8 @@ class AOTExecutorCodegen : public ExprVisitor { lowered_funcs_[target->str()] = IRModule(Map({})); } lowered_funcs_[target->str()]->Update(lowered_func->funcs); + // Update function metadata via looking at all primfuncs + UpdateFunctionMetadata(lowered_func, func, target); // Generate the TIR function call CreateFuncCall(GetRef(op), lowered_func->func_name); @@ -488,6 +566,8 @@ class AOTExecutorCodegen : public ExprVisitor { std::unordered_map sids_table_; /*! \brief lowered funcs */ std::unordered_map lowered_funcs_; + /*! \brief lowered funcs */ + Map function_metadata_; /*! \brief compile engine */ CompileEngine compile_engine_; /*! \brief the set of statements that make the program */ @@ -531,6 +611,7 @@ class AOTExecutorCodegen : public ExprVisitor { VisitExpr(func->body); auto prim_func = CreateMainFunc(func->params.size()); + UpdateMainWorkspaceSize(prim_func, func); LoweredOutput ret; ret.params = std::unordered_map>(); @@ -559,7 +640,7 @@ class AOTExecutorCodegen : public ExprVisitor { symbol_map.Set(GlobalVar(::tvm::runtime::symbol::tvm_run_func_prefix), prim_func); ret.lowered_funcs.Set(target_host_str, IRModule(symbol_map)); } - + ret.function_metadata = std::move(function_metadata_); ret.metadata = runtime::Metadata(input_vars_.size(), return_sid_.size(), runtime::kTvmExecutorAot); return ret; @@ -602,6 +683,10 @@ class AOTExecutorCodegenModule : public runtime::ModuleNode { } else if (name == "get_external_modules") { return PackedFunc( [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = get_external_modules(); }); + } else if (name == "get_function_metadata") { + return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { + *rv = this->output_.function_metadata; + }); } else if (name == "get_metadata") { return PackedFunc( [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = output_.metadata; }); diff --git a/src/relay/backend/build_module.cc b/src/relay/backend/build_module.cc index 88faff22cd31..880407f14b8e 100644 --- a/src/relay/backend/build_module.cc +++ b/src/relay/backend/build_module.cc @@ -62,6 +62,10 @@ struct ExecutorCodegen { virtual void UpdateOutput(BuildOutput* ret) = 0; + Map GetFunctionMetadata() { + return CallFunc>("get_function_metadata", nullptr); + } + std::unordered_map GetParams() { std::unordered_map ret; auto names = CallFunc>("list_params_name", nullptr); @@ -197,6 +201,10 @@ class RelayBuildModule : public runtime::ModuleNode { return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->executor_codegen_->GetExternalModules(); }); + } else if (name == "get_function_metadata") { + return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { + *rv = this->executor_codegen_->GetFunctionMetadata(); + }); } else if (name == "optimize") { return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { ICHECK_EQ(args.num_args, 2); diff --git a/src/relay/backend/graph_executor_codegen.cc b/src/relay/backend/graph_executor_codegen.cc index 6d3b93e08e61..ddcdeaac5d61 100644 --- a/src/relay/backend/graph_executor_codegen.cc +++ b/src/relay/backend/graph_executor_codegen.cc @@ -28,6 +28,8 @@ #include #include #include +#include +#include #include #include @@ -182,9 +184,96 @@ class GraphExecutorCodegen : public backend::MemoizedExprTranslator> + std::unordered_map> sid_workspace; + // This is a Map + std::unordered_map device_io; + // This is a Map + std::unordered_map device_consts; + + // Initialize the maps to zero + for (const auto& kv : storage_device_map_) { + auto sids = kv.second[0]; + auto devices = kv.second[1]; + CHECK_EQ(sids.size(), devices.size()); + for (uint32_t i = 0; i < sids.size(); i++) { + sid_workspace[devices[i]][sids[i]] = 0; + device_io[devices[i]] = 0; + device_consts[devices[i]] = 0; + } + } + + // Collect sizes of tensors + for (const auto& kv : storage_device_map_) { + auto size_bytes = CalculateRelayExprSizeBytes(kv.first->checked_type()); + auto sids = kv.second[0]; + auto devices = kv.second[1]; + if (kv.first->IsInstance()) { + for (const auto& dev : devices) { + device_consts[dev] += size_bytes; + } + continue; + } else if (kv.first->IsInstance() || kv.first == func->body) { + for (const auto& dev : devices) { + device_io[dev] += size_bytes; + } + continue; + } + for (uint32_t i = 0; i < sids.size(); i++) { + // Here we record the largest size of the tensor + // that share the same storage id, because storage_id will + // be shared between multiple tensors that are not live simultaneously. + if (size_bytes > sid_workspace[devices[i]][sids[i]]) { + sid_workspace[devices[i]][sids[i]] = size_bytes; + } + } + } + + // This is a Map + std::unordered_map device_workspace; + // Once we know the sizes of sids, we need to accumulate per device + for (const auto& dev_sid_size : sid_workspace) { + auto dev = dev_sid_size.first; + device_workspace[dev] = 0; + for (const auto& sid_size : dev_sid_size.second) { + device_workspace[dev] += sid_size.second; + } + } + + // Populate FunctionInfo + auto fi_node = make_object(); + // Initialize all target workspaces to zero + for (const auto& kv : targets_) { + auto tgt = kv.second; + fi_node->workspace_sizes.Set(tgt, 0); + } + for (const auto& dev_and_size : device_workspace) { + auto tgt = GetTargetFromInteger(dev_and_size.first); + fi_node->workspace_sizes.Set(tgt, dev_and_size.second); + fi_node->relay_primfuncs.Set(tgt, func); + } + for (const auto& dev_and_size : device_io) { + auto tgt = GetTargetFromInteger(dev_and_size.first); + fi_node->io_sizes.Set(tgt, dev_and_size.second); + } + for (const auto& dev_and_size : device_consts) { + auto tgt = GetTargetFromInteger(dev_and_size.first); + fi_node->constant_sizes.Set(tgt, dev_and_size.second); + } + + function_metadata_.Set(String(runtime::symbol::tvm_module_main), FunctionInfo(fi_node)); + } + LoweredOutput Codegen(relay::Function func) { auto pf = GetPackedFunc("relay.backend.GraphPlanMemory"); storage_device_map_ = (*pf)(func); + UpdateMainWorkspaceSize(func); // First we convert all the parameters into input nodes. for (auto param : func->params) { auto node_ptr = GraphInputNode::make_node_ptr(param->name_hint(), GraphAttrs()); @@ -212,6 +301,7 @@ class GraphExecutorCodegen : public backend::MemoizedExprTranslatorLowerExternalFunctions(); + ret.function_metadata = std::move(function_metadata_); return ret; } @@ -352,6 +442,75 @@ class GraphExecutorCodegen : public backend::MemoizedExprTranslator(); + for (const auto& kv : cfunc->funcs->functions) { + auto primfunc = Downcast(kv.second); + Integer workspace_size = CalculateWorkspaceBytes(primfunc); + Target primfunc_target = relay_target; + if (primfunc->attrs->dict.count("target")) { + primfunc_target = Downcast(primfunc->attrs->dict["target"]); + } + fi_node->workspace_sizes.Set(primfunc_target, workspace_size); + // Calculating size for I/O + for (auto const& param : primfunc->params) { + auto p_shape = primfunc->buffer_map[param]->shape; + int num_of_elements = 1; + for (const auto& dim_index_expr : p_shape) { + if (dim_index_expr->IsInstance()) { + num_of_elements *= dim_index_expr.as()->value; + } else { + // If shape is dynamic, we cannot calculate workspace in compile time. + num_of_elements = 0; + } + } + int element_size = primfunc->buffer_map[param]->dtype.bytes(); + fi_node->io_sizes.Set(primfunc_target, element_size * num_of_elements); + } + fi_node->constant_sizes.Set(primfunc_target, 0); + fi_node->tir_primfuncs.Set(primfunc_target, primfunc); + fi_node->relay_primfuncs.Set(primfunc_target, relay_func); + } + function_metadata_.Set(cfunc->func_name, FunctionInfo(fi_node)); + } + std::vector VisitExpr_(const CallNode* op) override { Expr expr = GetRef(op); Function func; @@ -408,30 +567,18 @@ class GraphExecutorCodegen : public backend::MemoizedExprTranslatorvalue; + target = GetTargetFromInteger(call_dev_type); // Normal Relay Function - if (targets_.size() == 1) { - // homogeneous execution. - const auto& it = targets_.begin(); - target = (*it).second; - } else { - // heterogeneous execution. - std::string call_dev_name; - if (call_dev_type == 0) { - call_dev_name = "llvm"; - } else { - call_dev_name = runtime::DeviceName(call_dev_type); - } - if (targets_.count(call_dev_type) == 0) { - LOG(FATAL) << "No target is provided for device " << call_dev_name; - } - target = targets_[call_dev_type]; - } + CCacheKey key = (*pf0)(func, target); CachedFunc lowered_func = (*pf1)(compile_engine_, key); if (!lowered_funcs_.count(target->str())) { lowered_funcs_[target->str()] = IRModule(Map({})); } lowered_funcs_[target->str()]->Update(lowered_func->funcs); + + // Update function metadata via looking at all primfuncs + UpdateFunctionMetadata(lowered_func, func, target); return GraphAddCallNode(op, _GetUniqueName(lowered_func->func_name), lowered_func->func_name, attrs); } @@ -577,6 +724,8 @@ class GraphExecutorCodegen : public backend::MemoizedExprTranslator> storage_device_map_; /*! \brief lowered funcs */ std::unordered_map lowered_funcs_; + /*! \brief lowered funcs */ + Map function_metadata_; /*! \brief name map */ std::unordered_map name_map_; /*! \brief compile engine */ @@ -643,6 +792,10 @@ class GraphExecutorCodegenModule : public runtime::ModuleNode { } else if (name == "get_metadata") { return PackedFunc( [sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { *rv = this->output_.metadata; }); + } else if (name == "get_function_metadata") { + return PackedFunc([sptr_to_self, this](TVMArgs args, TVMRetValue* rv) { + *rv = this->output_.function_metadata; + }); } else { return PackedFunc([](TVMArgs args, TVMRetValue* rv) {}); } diff --git a/src/relay/backend/utils.cc b/src/relay/backend/utils.cc new file mode 100644 index 000000000000..be811961e4a1 --- /dev/null +++ b/src/relay/backend/utils.cc @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/*! + * \file relay/backend/util.cc + * \brief Relay backend utilities. + */ + +#include "utils.h" + +namespace tvm { +namespace relay { +namespace backend { + +int64_t CalculateRelayExprSizeBytes(const Type& expr_type) { + if (expr_type->IsInstance()) { + auto tuple_type = Downcast(expr_type); + int64_t size = 0; + for (const auto& field : tuple_type->fields) { + size += CalculateRelayExprSizeBytes(field); + } + return size; + } + auto tensor_type = expr_type.as(); + auto shape = tensor_type->shape; + int num_of_elements = 1; + for (const auto& dim_index_expr : shape) { + if (dim_index_expr->IsInstance()) { + num_of_elements *= dim_index_expr.as()->value; + } else { + // If shape is dynamic, we cannot calculate workspace in compile time. + num_of_elements = 0; + } + } + auto element_size = tensor_type->dtype.bytes(); + return element_size * num_of_elements; +} + +TVM_REGISTER_NODE_TYPE(FunctionInfoNode); + +} // namespace backend +} // namespace relay +} // namespace tvm diff --git a/src/relay/backend/utils.h b/src/relay/backend/utils.h index c804768c99af..4f7cbde5b62c 100644 --- a/src/relay/backend/utils.h +++ b/src/relay/backend/utils.h @@ -46,6 +46,37 @@ namespace tvm { namespace relay { namespace backend { +struct FunctionInfoNode : public Object { + Map workspace_sizes; + Map io_sizes; + Map constant_sizes; + Map tir_primfuncs; + Map relay_primfuncs; + + void VisitAttrs(tvm::AttrVisitor* v) { + v->Visit("workspace_sizes", &workspace_sizes); + v->Visit("io_sizes", &io_sizes); + v->Visit("constant_sizes", &constant_sizes); + v->Visit("tir_primfuncs", &tir_primfuncs); + v->Visit("relay_primfuncs", &relay_primfuncs); + } + + static constexpr const char* _type_key = "relay.backend.FunctionInfo"; + TVM_DECLARE_FINAL_OBJECT_INFO(FunctionInfoNode, Object); +}; + +class FunctionInfo : public ObjectRef { + public: + TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(FunctionInfo, ObjectRef, FunctionInfoNode); +}; + +/*! + * \brief Calculate the storage required to store the type of relay.Expr + * + * \param func The relay expr for which the storage is calculated + */ +int64_t CalculateRelayExprSizeBytes(const Type& expr_type); + /*! * \brief Executor generator artifacts. Those artifacts are subsequently * used by the relay build process. @@ -54,6 +85,7 @@ struct LoweredOutput { std::string graph_json; Map lowered_funcs; Array external_mods; + Map function_metadata; std::unordered_map> params; runtime::Metadata metadata; }; diff --git a/src/tir/analysis/calculate_workspace.cc b/src/tir/analysis/calculate_workspace.cc index 8b42efb12ccd..2f5f5e3a671c 100644 --- a/src/tir/analysis/calculate_workspace.cc +++ b/src/tir/analysis/calculate_workspace.cc @@ -50,7 +50,12 @@ size_t WorkspaceCalculator::CalculateExtentsSize(const AllocateNode* op) { size_t element_size_bytes = op->dtype.bytes(); size_t num_elements = 1; for (const auto& ext : op->extents) { - num_elements *= Downcast(ext)->value; + if (ext->IsInstance()) { + num_elements *= Downcast(ext)->value; + } else { + // We cant statically calculate workspace for dynamic shapes + num_elements = 0; + } } return num_elements * element_size_bytes; } diff --git a/tests/python/unittest/test_micro_model_library_format.py b/tests/python/unittest/test_micro_model_library_format.py index 712bd8d348a2..d2c519da22b5 100644 --- a/tests/python/unittest/test_micro_model_library_format.py +++ b/tests/python/unittest/test_micro_model_library_format.py @@ -45,9 +45,16 @@ def validate_graph_json(extract_dir, factory): @tvm.testing.requires_micro -def test_export_model_library_format_c(): +@pytest.mark.parametrize( + "target", + [ + ("graph", tvm.target.target.micro("host")), + ("aot", tvm.target.target.micro("host", options="-executor=aot")), + ], +) +def test_export_model_library_format_c(target): + executor, _target = target with utils.TempDirectory.set_keep_for_debug(True): - target = tvm.target.target.micro("host") with tvm.transform.PassContext(opt_level=3, config={"tir.disable_vectorize": True}): relay_mod = tvm.parser.fromtext( """ @@ -59,8 +66,8 @@ def @main(%a : Tensor[(1, 2), uint8], %b : Tensor[(1, 2), float32], %c : Tensor[ ) factory = tvm.relay.build( relay_mod, - target, - target_host=target, + _target, + target_host=_target, mod_name="add", params={"c": numpy.array([[2.0, 4.0]], dtype="float32")}, ) @@ -78,24 +85,41 @@ def @main(%a : Tensor[(1, 2), uint8], %b : Tensor[(1, 2), float32], %c : Tensor[ with open(os.path.join(extract_dir, "metadata.json")) as json_f: metadata = json.load(json_f) - assert metadata["version"] == 1 + assert metadata["version"] == 2 assert metadata["model_name"] == "add" export_datetime = datetime.datetime.strptime( metadata["export_datetime"], "%Y-%m-%d %H:%M:%SZ" ) assert (datetime.datetime.now() - export_datetime) < datetime.timedelta(seconds=60 * 5) - assert metadata["target"] == {"1": str(target)} - assert metadata["memory"] == [ - {"storage_id": 0, "size_bytes": 2, "input_binding": "a"}, - {"storage_id": 1, "size_bytes": 8, "input_binding": "b"}, - {"storage_id": 2, "size_bytes": 8, "input_binding": "p0"}, - {"storage_id": 3, "size_bytes": 8}, + assert metadata["target"] == {"1": str(_target)} + if executor == "graph": + assert metadata["memory"]["sids"] == [ + {"storage_id": 0, "size_bytes": 2, "input_binding": "a"}, + {"storage_id": 1, "size_bytes": 8, "input_binding": "b"}, + {"storage_id": 2, "size_bytes": 8, "input_binding": "p0"}, + {"storage_id": 3, "size_bytes": 8}, + ] + assert metadata["memory"]["functions"]["main"] == [ + { + "constants_size_bytes": 8, + "device": 1, + "io_size_bytes": 18, + "workspace_size_bytes": 0, + } + ] + assert metadata["memory"]["functions"]["operator_functions"][0]["workspace"] == [ + {"device": 1, "workspace_size_bytes": 0} ] + assert ( + "fused_cast_multiply_add" + in metadata["memory"]["functions"]["operator_functions"][0]["function_name"] + ) assert os.path.exists(os.path.join(extract_dir, "codegen", "host", "src", "lib0.c")) assert os.path.exists(os.path.join(extract_dir, "codegen", "host", "src", "lib1.c")) - validate_graph_json(extract_dir, factory) + if executor == "graph": + validate_graph_json(extract_dir, factory) with open(os.path.join(extract_dir, "relay.txt")) as relay_f: assert relay_f.read() == str(relay_mod) @@ -141,19 +165,34 @@ def @main(%a : Tensor[(1, 2), uint8], %b : Tensor[(1, 2), float32], %c : Tensor[ with open(os.path.join(extract_dir, "metadata.json")) as json_f: metadata = json.load(json_f) - assert metadata["version"] == 1 + assert metadata["version"] == 2 assert metadata["model_name"] == "add" export_datetime = datetime.datetime.strptime( metadata["export_datetime"], "%Y-%m-%d %H:%M:%SZ" ) assert (datetime.datetime.now() - export_datetime) < datetime.timedelta(seconds=60 * 5) assert metadata["target"] == {"1": str(target)} - assert metadata["memory"] == [ + assert metadata["memory"]["sids"] == [ {"storage_id": 0, "size_bytes": 2, "input_binding": "a"}, {"storage_id": 1, "size_bytes": 8, "input_binding": "b"}, {"storage_id": 2, "size_bytes": 8, "input_binding": "p0"}, {"storage_id": 3, "size_bytes": 8}, ] + assert metadata["memory"]["functions"]["main"] == [ + { + "constants_size_bytes": 8, + "device": 1, + "io_size_bytes": 18, + "workspace_size_bytes": 0, + } + ] + assert metadata["memory"]["functions"]["operator_functions"][0]["workspace"] == [ + {"device": 1, "workspace_size_bytes": 0} + ] + assert ( + "fused_cast_multiply_add" + in metadata["memory"]["functions"]["operator_functions"][0]["function_name"] + ) assert os.path.exists(os.path.join(extract_dir, "codegen", "host", "lib", "lib0.o")) @@ -167,11 +206,73 @@ def @main(%a : Tensor[(1, 2), uint8], %b : Tensor[(1, 2), float32], %c : Tensor[ assert "p0" in params +@tvm.testing.requires_micro +@pytest.mark.parametrize( + "target", + [ + ("graph", tvm.target.target.micro("host")), + ("aot", tvm.target.target.micro("host", options="-executor=aot")), + ], +) +def test_export_model_library_format_workspace(target): + executor, _target = target + with tvm.transform.PassContext(opt_level=3, config={"tir.disable_vectorize": True}): + relay_mod = tvm.parser.fromtext( + """ + #[version = "0.0.5"] + def @main(%p0: Tensor[(1, 56, 56, 128), int16], %p1: Tensor[(3, 3, 128, 1), int16], %p2: Tensor[(1, 1, 1, 128), int32]){ + %0 = nn.conv2d(%p0, %p1, padding=[1, 1, 1, 1], groups=128, channels=128, kernel_size=[3, 3], data_layout="NHWC", kernel_layout="HWOI", out_dtype="int32") /* ty=Tensor[(1, 56, 56, 128), int32] */; + %1 = add(%0, %p2) /* ty=Tensor[(1, 56, 56, 128), int32] */; + %2 = fixed_point_multiply(%1, multiplier=2080045879, shift=-4) /* ty=Tensor[(1, 56, 56, 128), int32] */; + %3 = clip(%2, a_min=0f, a_max=255f) /* ty=Tensor[(1, 56, 56, 128), int32] */; + cast(%3, dtype="uint8") /* ty=Tensor[(1, 56, 56, 128), uint8] */ + } + """ + ) + factory = tvm.relay.build(relay_mod, _target, target_host=_target, mod_name="qnn_conv2d") + + temp_dir = utils.tempdir() + mlf_tar_path = temp_dir.relpath("lib.tar") + import tvm.micro as micro + + micro.export_model_library_format(factory, mlf_tar_path) + tf = tarfile.open(mlf_tar_path) + + extract_dir = temp_dir.relpath("extract") + os.mkdir(extract_dir) + tf.extractall(extract_dir) + + with open(os.path.join(extract_dir, "metadata.json")) as json_f: + metadata = json.load(json_f) + assert metadata["version"] == 2 + assert metadata["model_name"] == "qnn_conv2d" + export_datetime = datetime.datetime.strptime( + metadata["export_datetime"], "%Y-%m-%d %H:%M:%SZ" + ) + assert (datetime.datetime.now() - export_datetime) < datetime.timedelta(seconds=60 * 5) + assert metadata["target"] == {"1": str(_target)} + assert metadata["memory"]["functions"]["main"] == [ + { + "constants_size_bytes": 0, + "device": 1, + "io_size_bytes": 1207040, + "workspace_size_bytes": 2466816, + } + ] + assert metadata["memory"]["functions"]["operator_functions"][0]["workspace"] == [ + {"device": 1, "workspace_size_bytes": 2466816} + ] + assert ( + "fused_nn_conv2d_add_fixed_point_multiply_clip_cast" + in metadata["memory"]["functions"]["operator_functions"][0]["function_name"] + ) + + @tvm.testing.requires_micro def test_export_model(): module = tvm.support.FrontendTestModule() factory = executor_factory.GraphExecutorFactoryModule( - None, tvm.target.target.micro("host"), '"graph_json"', module, "test_module", {} + None, tvm.target.target.micro("host"), '"graph_json"', module, "test_module", {}, {} ) temp_dir = utils.tempdir()