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

[GitHub] [tvm] junrushao1994 commented on a change in pull request #7534: [Target] Add support for target object with host field compatible with previous api

junrushao1994 commented on a change in pull request #7534:
URL: https://github.com/apache/tvm/pull/7534#discussion_r592000534



##########
File path: python/tvm/driver/build_module.py
##########
@@ -409,6 +419,16 @@ def build(inputs, args=None, target=None, target_host=None, name="default_functi
     if not target_host:
         target_host = "llvm" if tvm.runtime.enabled("llvm") else "stackvm"
 
+    new_input_mod = {}
+    for tar, mod in target_input_mod.items():

Review comment:
       ditto
   
   ```suggestion
       for tgt, mod in target_input_mod.items():
   ```

##########
File path: python/tvm/driver/build_module.py
##########
@@ -399,8 +399,18 @@ def build(inputs, args=None, target=None, target_host=None, name="default_functi
         if not isinstance(mod, tvm.IRModule):
             raise ValueError("inputs must be Schedule, IRModule," "or dict of str to IRModule.")
 
+    new_input_mod = {}
+    for tar, mod in target_input_mod.items():

Review comment:
       `tar` might mean something else
   
   ```suggestion
       for tgt, mod in target_input_mod.items():
   ```

##########
File path: python/tvm/relay/build_module.py
##########
@@ -129,6 +129,15 @@ def build(self, mod, target=None, target_host=None, params=None):
         old_autotvm_silent = autotvm.GLOBAL_SCOPE.silent
         autotvm.GLOBAL_SCOPE.silent = use_auto_scheduler
 
+        # Assume the target host of all targets in heterogenous target are identical
+        if isinstance(target, dict):
+            for k in target:
+                target[k] = Target(target[k], target_host)
+                target_host = target[k].host

Review comment:
       ditto

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -256,8 +256,11 @@ class VMFunctionCompiler : ExprFunctor<void(const Expr& expr)> {
         target_host_(target_host),
         expr_device_map_(std::move(expr_device_map)) {
     for (const auto& it : targets) {
+      targets.Set(it.first, Target(targets[it.first], target_host));
+      target_host = targets[it.first]->GetHost().value_or(Target());
       targets_[it.first->value] = it.second;
     }

Review comment:
       It is a bit tricky to mutate the container while iterating it. It is quite error-prone (because tvm::Map does copy-on-write internally). Please consider reconstruct a Map instead. Also, the logic is all over the codebase, so probably we need a helper function.

##########
File path: src/relay/backend/build_module.cc
##########
@@ -235,8 +235,15 @@ class RelayBuildModule : public runtime::ModuleNode {
    * \param target_host Host target device
    */
   void Build(IRModule mod, const TargetsMap& targets, const tvm::Target& target_host) {
+    // Create protected variable targets_ from ground up
     targets_ = targets;
     target_host_ = target_host;
+    for (const auto& it : targets) {
+      // Construct a new target with target host filed if available
+      targets_.Set(it.first, Target(it.second, target_host_));
+      target_host_ = targets_[it.first]->GetHost().value_or(Target());
+    }

Review comment:
       ditto

##########
File path: src/relay/backend/build_module.cc
##########
@@ -481,6 +488,13 @@ class RelayBuildModule : public runtime::ModuleNode {
     const runtime::PackedFunc* pf = runtime::Registry::Get("codegen.LLVMModuleCreate");
     if (!target_host.defined()) target_host = (pf != nullptr) ? Target("llvm") : Target("stackvm");
 
+    // Update all the targets in the _targets TargetsMap
+    for (const auto& it : targets_) {
+      // Construct a new target with target host filed if available
+      targets_.Set(it.first, Target(it.second, target_host));
+      target_host = targets_[it.first]->GetHost().value_or(Target());
+    }

Review comment:
       ditto

##########
File path: python/tvm/relay/backend/vm.py
##########
@@ -65,6 +65,13 @@ def compile(mod, target=None, target_host=None, params=None):
     compiler = VMCompiler()
     if params:
         compiler.set_params(params)
+    if isinstance(target, dict):
+        for k in target:
+            target[k] = tvm.target.Target(target[k], target_host)
+            target_host = target[k].host

Review comment:
       I saw the lookup, so it is probably better to use `.item` here
   
   ```suggestion
           for tgt, mod in target.items():
   ```

##########
File path: python/tvm/driver/tvmc/autotuner.py
##########
@@ -23,6 +23,8 @@
 
 from urllib.parse import urlparse
 
+import tvm

Review comment:
       Don't need to import the entire package
   
   ```suggestion
   from tvm.target import Target
   ```

##########
File path: src/auto_scheduler/feature.cc
##########
@@ -1397,9 +1397,12 @@ void GetPerStoreFeaturesFromFile(const std::string& filename, int max_lines, int
     if (find_res == task_cache.end()) {
       // rebuild task
       Array<te::Tensor> tensors = (*workload_key_to_tensors)(workload_key);
-      task = SearchTask(ComputeDAG(tensors), workload_key, cur_inp->task->target,
-                        cur_inp->task->target_host, cur_inp->task->hardware_params,
-                        cur_inp->task->layout_rewrite_option, cur_inp->task->task_input_names);
+      Target target = cur_inp->task->target, target_host = cur_inp->task->target_host;

Review comment:
       split into two lines

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -900,6 +903,10 @@ void VMCompiler::Lower(IRModule mod, const TargetsMap& targets, const tvm::Targe
   exec_ = make_object<Executable>();
   targets_ = targets;
   target_host_ = target_host;
+  for (auto& iter : targets_) {
+    targets_.Set(iter.first, Target(targets_[iter.first], target_host_));
+    target_host_ = targets[iter.first]->GetHost().value_or(Target());
+  }

Review comment:
       ditto

##########
File path: src/relay/backend/vm/compiler.cc
##########
@@ -1001,8 +1008,14 @@ transform::Sequential MemoryOpt(tvm::Target host_target, TargetsMap targets) {
   return transform::Sequential(pass_seqs);
 }
 
-IRModule VMCompiler::OptimizeModule(IRModule mod, const TargetsMap& targets,
-                                    const Target& target_host) {
+IRModule VMCompiler::OptimizeModule(IRModule mod, const TargetsMap& targets_arg,
+                                    const Target& target_host_arg) {
+  TargetsMap targets = targets_arg;
+  Target target_host = target_host_arg;
+  for (auto& iter : targets) {
+    targets.Set(iter.first, Target(targets[iter.first], target_host));
+    target_host = targets[iter.first]->GetHost().value_or(Target());
+  }

Review comment:
       ditto

##########
File path: src/target/target.cc
##########
@@ -375,7 +375,7 @@ Target::Target(const Map<String, ObjectRef>& config) {
 
 Target::Target(Target target, Target host) {
   ObjectPtr<TargetNode> n = make_object<TargetNode>(*target.get());
-  CHECK(!n->host.defined())
+  CHECK((!n->host.defined()) || n->host == host)

Review comment:
       ```suggestion
     CHECK(!n->host.defined() || n->host == host)
   ```

##########
File path: src/relay/transforms/memory_alloc.cc
##########
@@ -458,6 +462,10 @@ Pass ManifestAlloc(Target target_host, Map<tvm::Integer, tvm::Target> targets) {
 
 TVM_REGISTER_GLOBAL("relay.transform.ManifestAlloc")
     .set_body_typed([](Target target_host, Map<tvm::Integer, tvm::Target> targets) {
+      for (auto& iter : targets) {
+        targets.Set(iter.first, Target(targets[iter.first], target_host));
+        target_host = targets[iter.first]->GetHost().value_or(Target());
+      }

Review comment:
       ditto

##########
File path: src/driver/driver_api.cc
##########
@@ -185,9 +185,11 @@ IRModule lower(te::Schedule sch, const Array<te::Tensor>& args, const std::strin
   return mod;
 }
 
-std::pair<IRModule, IRModule> SplitDevHostFuncs(IRModule mod_mixed, const Target& target,
-                                                const Target& target_host,
+std::pair<IRModule, IRModule> SplitDevHostFuncs(IRModule mod_mixed, const Target& target_arg,
+                                                const Target& target_host_arg,
                                                 const transform::PassContext& pass_ctx) {
+  Target target = Target(target_arg, target_host_arg),
+         target_host = target->GetHost().value_or(Target());

Review comment:
       split into two statements

##########
File path: python/tvm/relay/backend/vm.py
##########
@@ -130,6 +137,15 @@ def lower(self, mod, target=None, target_host=None):
         """
         target = self._update_target(target)
         target_host = self._update_target_host(target, target_host)
+
+        if isinstance(target, dict):
+            for k in target:
+                target[k] = tvm.target.Target(target[k], target_host)
+                target_host = target[k].host

Review comment:
       ditto

##########
File path: python/tvm/relay/build_module.py
##########
@@ -263,14 +272,20 @@ def build(ir_mod, target=None, target_host=None, params=None, mod_name="default"
             "instead of deprecated parameter mod (tvm.relay.function.Function)",
             DeprecationWarning,
         )
-
     target = _update_target(target)
-
     if isinstance(target_host, (str, Target)):
         target_host = Target(target_host)
     elif target_host:
         raise ValueError("target host must be the type of str, " + "tvm.target.Target, or None")
 
+    if isinstance(target, dict):
+        for k in target:
+            target[k] = Target(target[k], target_host)
+            target_host = target[k].host

Review comment:
       ditto

##########
File path: python/tvm/relay/backend/vm.py
##########
@@ -167,6 +183,15 @@ def optimize(self, mod, target=None, target_host=None, params=None):
         """
         target = self._update_target(target)
         target_host = self._update_target_host(target, target_host)
+
+        if isinstance(target, dict):
+            for k in target:
+                target[k] = tvm.target.Target(target[k], target_host)
+                target_host = target[k].host

Review comment:
       I saw the same logic all over this file. would you like to add a helper function?

##########
File path: src/relay/transforms/memory_alloc.cc
##########
@@ -415,6 +415,10 @@ class DialectRewriter : public ExprMutator {
 namespace transform {
 
 Pass ManifestAlloc(Target target_host, Map<tvm::Integer, tvm::Target> targets) {
+  for (auto& iter : targets) {
+    targets.Set(iter.first, Target(targets[iter.first], target_host));
+    target_host = targets[iter.first]->GetHost().value_or(Target());
+  }

Review comment:
       ditto

##########
File path: src/driver/driver_api.cc
##########
@@ -253,31 +255,45 @@ std::pair<IRModule, IRModule> SplitDevHostFuncs(IRModule mod_mixed, const Target
 }
 
 // Build for heterogeneous execution.
-runtime::Module build(const Map<Target, IRModule>& inputs, const Target& target_host) {
+runtime::Module build(const Map<Target, IRModule>& inputs, const Target& target_host_arg) {
   auto pass_ctx = transform::PassContext::Current();
 
   std::vector<runtime::Module> device_modules;
-  Target target_host_val = target_host;
+  Target target_host = target_host_arg;
+  Map<Target, IRModule> updated_inputs;
+
+  // Fetch previous defined target host in targets
+  for (const auto& it : inputs) {
+    auto target = Target(it.first, target_host);
+    target_host = target->GetHost().value_or(Target());
+  }
+
   if (!target_host.defined()) {
     for (const auto& it : inputs) {
       if (it.first->kind->device_type == kDLCPU || it.first->kind->device_type == kDLMicroDev) {
-        target_host_val = it.first;
+        target_host = it.first;
         break;
       }
     }
   }
 
-  if (!target_host_val.defined()) {
-    target_host_val = DefaultTargetHost(target_host_val);
+  if (!target_host.defined()) {
+    target_host = DefaultTargetHost(target_host);
+  }
+
+  // Update target host for all targets
+  for (const auto& it : inputs) {
+    auto target = Target(it.first, target_host);
+    updated_inputs.Set(target, it.second);
   }

Review comment:
       ditto

##########
File path: tests/micro/qemu/test_zephyr.py
##########
@@ -46,7 +46,7 @@
 def _make_sess_from_op(model, zephyr_board, west_cmd, op_name, sched, arg_bufs):
     target = tvm.target.target.micro(model)
     with tvm.transform.PassContext(opt_level=3, config={"tir.disable_vectorize": True}):
-        mod = tvm.build(sched, arg_bufs, target, target_host=target, name=op_name)
+        mod = tvm.build(sched, arg_bufs, tvm.target.Target(target, target), name=op_name)

Review comment:
       move the Target construction to the beginning of the function

##########
File path: src/target/target.cc
##########
@@ -408,12 +408,17 @@ Map<String, ObjectRef> TargetNode::Export() const {
       {"tag", this->tag},
       {"keys", this->keys},
   };
+  if (this->host.defined()) result.Set("host", this->GetHost().value_or(Target())->Export());

Review comment:
       ```suggestion
     if (this->host.defined()) {
       result.Set("host", Downcast<Target>(this->host)->Export());
     }
   ```

##########
File path: src/driver/driver_api.cc
##########
@@ -304,21 +320,26 @@ runtime::Module build(const Map<Target, IRModule>& inputs, const Target& target_
 }
 
 // Build for heterogeneous execution when target is a string.
-runtime::Module build(const Map<String, IRModule>& inputs, const Target& target_host) {
-  Map<Target, IRModule> updated_input;
-  for (const auto& it : inputs) {
-    auto target = Target(it.first);
+runtime::Module build(const Map<String, IRModule>& inputs_arg, const Target& target_host_arg) {
+  Map<Target, IRModule> updated_inputs;
+  Target target_host = target_host_arg;
+  for (const auto& it : inputs_arg) {
+    auto target = Target(Target(it.first), target_host);
+    target_host = target->GetHost().value_or(Target());
     Optional<String> device = target->GetAttr<String>("device");
     if (device.defined() && device.value() == "vta") {
       target = Target("ext_dev");
     }
-    updated_input.Set(target, it.second);
+    updated_inputs.Set(target, it.second);
   }
-  return build(updated_input, target_host);
+  return build(updated_inputs, target_host);
 }
 
 // Build for homogeneous execution.
-runtime::Module build(const IRModule& funcs, const Target& target, const Target& target_host) {
+runtime::Module build(const IRModule& funcs, const Target& target_arg,
+                      const Target& target_host_arg) {
+  auto target = Target(target_arg, target_host_arg),
+       target_host = target->GetHost().value_or(Target());

Review comment:
       split into two statements

##########
File path: tutorials/get_started/tensor_expr_get_started.py
##########
@@ -154,7 +153,7 @@
 # - fadd runs the actual computation.
 # - asnumpy() copies the GPU array back to the CPU and we can use this to verify correctness
 #
-ctx = tvm.context(tgt, 0)
+ctx = tvm.context(str(tgt), 0)

Review comment:
       ```suggestion
   ctx = tvm.context(tgt.kind.name, 0)
   ```

##########
File path: tests/python/integration/test_tuning.py
##########
@@ -131,12 +131,11 @@ def teardown_module():
 
 
 def get_sample_task(target=tvm.target.cuda(), target_host=None):
+    target = tvm.target.Target(target, target_host)
+    target_host = target.host

Review comment:
       is this used?




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