You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2022/05/04 15:36:16 UTC

[GitHub] [tvm] Lunderberg commented on a diff in pull request #11173: [Relay] Support 'external codegen targets'.

Lunderberg commented on code in PR #11173:
URL: https://github.com/apache/tvm/pull/11173#discussion_r864958422


##########
src/target/compilation_config.cc:
##########
@@ -53,194 +73,179 @@ VirtualDevice CompilationConfigNode::CanonicalVirtualDevice(
                                                     target, virtual_device->memory_scope));
 }
 
-void CompilationConfigNode::EstablishDefaultVirtualDevices(const transform::PassContext& pass_ctx) {
+void CompilationConfigNode::Init(const transform::PassContext& pass_ctx,
+                                 const Array<Target>& raw_targets) {
+  VLOG_CONTEXT << "CompilationConfig";
+  CHECK_GT(raw_targets.size(), 0U) << "Require at least one target";
+
   //
-  // Gather the hints as to what our default device type for the 'host' should be, and
-  // create an appropriate target if we don't already have one.
+  // Decide on the host target.
   //
-  DLDeviceType host_device_type;
-  if (host_target.defined()) {
-    CHECK(!host_target->host.defined()) << "Host targets are not expected to have hosts";
-    host_device_type = static_cast<DLDeviceType>(host_target->kind->device_type);
-    VLOG(1) << "Using the given host target " << host_target->ToDebugString() << " of device type "
-            << host_device_type << " for the host target";
-    for (const auto& primitive_target : primitive_targets) {
-      if (primitive_target->host.defined() &&
-          !StructuralEqual()(primitive_target->host, host_target)) {
-        VLOG(1) << "The primitive target " << primitive_target->ToDebugString()
-                << " already has a host which disagrees with the desired host target. It "
-                << "will be ignored.";
-      }
-    }
-  } else if (primitive_targets.size() == 1 && primitive_targets.front()->host.defined()) {
-    host_target = primitive_targets.front()->GetHost().value();
-    CHECK(!host_target->host.defined()) << "Host targets are not expected to have hosts";
-    host_device_type = static_cast<DLDeviceType>(host_target->kind->device_type);
-    VLOG(1) << "Using the host of the unique primitive target, namely "
-            << host_target->ToDebugString() << " of device type " << host_device_type
-            << " for the host target";
-  } else if (primitive_targets.size() == 1 &&
-             primitive_targets.front()->kind->device_type == kDLCPU) {
-    // In the homogenous case without an explicit host target just use the given target so long as
-    // it's a CPU.
-    host_device_type = kDLCPU;
-    host_target = primitive_targets.front();
-    VLOG(1) << "Using the unique primitive target " << host_target->ToDebugString()
-            << " of device type " << host_device_type << " for the host target";
+
+  // Any CPU-like targets?
+  auto cpu_itr = std::find_if(raw_targets.begin(), raw_targets.end(), [](const Target& target) {
+    // TODO(tvm-team): AoT only works with kDLCPU device type. We can remove kDLHexagon
+    // here once we refactored kDLHexagon to kDLCPU.
+    return target->kind->device_type == kDLCPU || target->kind->device_type == kDLHexagon;
+  });
+
+  // Any targets with a host?
+  auto has_host_itr = std::find_if(raw_targets.begin(), raw_targets.end(),
+                                   [](const Target& target) { return target->host.defined(); });
+
+  if (has_host_itr != raw_targets.end()) {
+    // RULE A: If any raw target has a host, use the first such host for all the primitive
+    // targets.
+    host_target = Target((*has_host_itr)->GetHost().value(), /*host=*/Target());
+    VLOG(1) << "The target " << (*has_host_itr)->ToDebugString() << " supplies a host target "
+            << host_target->ToDebugString() << " of device type " << host_target->kind->device_type;
+  } else if (cpu_itr != raw_targets.end()) {
+    // RULE B: If any raw target is for a CPU-like device then also use that as the host.
+    host_target = Target(*cpu_itr, /*host=*/Target());
+    VLOG(1) << "Using target " << host_target->ToDebugString() << " of CPU-like device type "
+            << host_target->kind->device_type << " as the host target";
   } else {
-    // Fallback.
-    host_device_type = kDLCPU;
-    // Even if the list of available targets already includes one for kDLCPU we won't use it
-    // in the hetrogeneous case since its options may not be appropriate for host code
-    // (eg shape functions). Instead, create a fresh default Target.
-    host_target = MakeDefaultTarget(host_device_type);
-    VLOG(1) << "Using the default target " << host_target->ToDebugString() << " of device type "
-            << host_device_type << " for the host target";
+    // RULE C: Otherwise, create a default CPU host target.
+    host_target = MakeDefaultCPUTarget();
+    VLOG(1) << "Created a default target " << host_target->ToDebugString() << " of device type "
+            << host_target->kind->device_type << " for the host target";
   }
   ICHECK(host_target.defined());
   ICHECK(!host_target->host.defined());
 
-  if (host_device_type != kDLCPU) {
-    // I think we're on thin ice here until we've audited the code base for assumed kDLCPU.
-    VLOG(1) << "The host target is not a CPU.";
+  if (host_target->kind->device_type != kDLCPU) {
+    // I think we're on thin ice here until we've audited the code base for assumed CPU hosts.
+    VLOG(1) << "The host target is not a CPU. This is probably not going to work.";
   }
 
   //
   // Establish the host VirtualDevice.
   //
-  host_virtual_device =
-      virtual_device_cache_.Unique(VirtualDevice(host_device_type,
-                                                 /*virtual_device_id=*/0, host_target));
+  host_virtual_device = virtual_device_cache_.Unique(
+      VirtualDevice(static_cast<DLDeviceType>(host_target->kind->device_type),
+                    /*virtual_device_id=*/0, host_target));
+  ICHECK(host_virtual_device.defined());
+  ICHECK(host_virtual_device->target.defined());
 
   //
-  // Now that we've settled on a host, we can set it as the host on all primitive targets.
+  // Now that we've settled on a host, we can set it as the host on all the raw targets.
   //
-  Array<Target> new_primitve_targets;
-  new_primitve_targets.reserve(primitive_targets.size());
-  for (const auto& primitive_target : primitive_targets) {
-    new_primitve_targets.push_back(Target(primitive_target, host_target));
+  primitive_targets.clear();
+  primitive_targets.reserve(raw_targets.size());
+  for (const auto& raw_target : raw_targets) {
+    if (raw_target->host.defined() && !StructuralEqual()(raw_target->host, host_target)) {
+      VLOG(1) << "The target " << raw_target->ToDebugString()
+              << " already has a host which disagrees with the desired host target. It "
+              << "will be overridden.";
+    }
+    primitive_targets.push_back(Target(raw_target, host_target));
   }
-  primitive_targets = new_primitve_targets;
+  ICHECK_GT(primitive_targets.size(), 0U);
 
   //
-  // Gather the hints as to what our default device type for primitives should be.
+  // Check the primitive_targets are ordered correctly re Target::IsExternalCodegenFor.
+  //
+  std::unordered_set<DLDeviceType> primitive_target_device_types;
+  for (const auto& target : primitive_targets) {
+    primitive_target_device_types.emplace(static_cast<DLDeviceType>(target->kind->device_type));
+  }
+  for (DLDeviceType device_type : primitive_target_device_types) {
+    Target first_primitive_target;
+    for (const auto& current_primitive_target : primitive_targets) {
+      if (current_primitive_target->kind->device_type != device_type) {
+        continue;
+      }
+      if (!first_primitive_target.defined()) {
+        first_primitive_target = current_primitive_target;
+        continue;
+      }
+      CHECK(current_primitive_target.IsExternalCodegenFor(first_primitive_target))

Review Comment:
   Is it legal for a device_type to have only a single external codegen, and if not, can we explicitly check for it?
   
   If it is legal to only a single external codegen, then the error message that would occur on the second of two external codegens would describe it as an issue with the second target, rather than the first.  If having a single external codegen is legal, but having two external codegens without a non-external codegen isn't, I think we should add the following check to point users to the first target as the issue.
   
   ```c++
   TargetKindAttrMap<Bool> attr_map = TargetKind::GetAttrMap<Bool>(::tvm::attr::kIsExternalCodegen);
   CHECK(!attr_map.get(that->kind, Bool(false)))
       << "If multiple targets were given for a device type, "
       << "the first target for that device type must be non-external.  "
       << "The first target for " << device_type
       << "was " << first_primitive_target->ToDebugString()
       << ", which is an external target.";
   ```



##########
include/tvm/target/compilation_config.h:
##########
@@ -74,16 +63,37 @@ class CompilationConfigNode : public Object {
   Target host_target;
 
   /*!
-   * \brief Vector of all available \p Targets for compiling primitive operators. May contain
-   * a \p Target for the same device type as for the \p host_target, however the \p host_target
-   * should be used for all host computations and data. Each \p Target will have \p host_target
-   * as its host.
+   * \brief Vector of all available \p Targets for partitioning or compiling primitive tensor
+   * operators (kernels). May contain a \p Target for the same device type as for the
+   * \p host_target, however the \p host_target should be used for all host computations and data.
+   * Each \p Target will have \p host_target as its 'host'.
+   *
+   * It is possible to have multiple primitive targets for the same device type. However given

Review Comment:
   From the description here and the `CompilationConfigNode::Init`, it looks like it is the user's responsibility to ensure that the `primitive_targets` array is sorted so that this condition holds.  However, the `TargetKind::IsExternalCodegenFor` already provides enough information to generate this ordering.  Is the ordering of the list used to specify anything else by the user?  If not, would there be any downsides to having `CompilationConfigNode::Init` sort the list of targets by `(target_kind, is_external)` to ensure this condition holds?



##########
src/relay/backend/te_compiler.cc:
##########
@@ -952,25 +919,24 @@ backend::FunctionInfo UpdateMainWorkspaceSize(const IRModule& mod, tec::TargetMa
   Map<Target, Function> relay_primfuncs;
 
   // Initialize all target workspaces to zero
-  for (const auto& kv : targets) {
-    auto tgt = kv.second;
-    workspace_sizes.Set(tgt, 0);
+  for (const auto& target : config->primitive_targets) {
+    workspace_sizes.Set(target, 0);
   }
 
   for (const auto& dev_and_size : device_workspace) {
-    auto tgt = tec::GetTargetFromInteger(dev_and_size.first, targets);
-    workspace_sizes.Set(tgt, dev_and_size.second);
-    relay_primfuncs.Set(tgt, func);
+    Target target = config->FindPrimitiveTargetOrFail(dev_and_size.first);

Review Comment:
   It looks like the main use of `FindPrimitiveTargetOrFail` is to de-duplicate targets that share resource pools.  What is the advantage of using the target itself to handle this de-duplication, rather than using `target->kind->device_type` as the key for tracking resource usage?



##########
src/target/compilation_config.cc:
##########
@@ -39,6 +38,27 @@ void CompilationConfigNode::VisitAttrs(AttrVisitor* v) {
   // NOTE: The virtual_device_cache_ is not accessible via FFI.
 }
 
+Target CompilationConfigNode::FindPrimitiveTargetOrFail(DLDeviceType device_type) const {
+  if (device_type < 0 && primitive_targets.size() == 1) {
+    // In the homogenous case don't be fussy with device types.

Review Comment:
   Should we validate that `device_type` matches the `primitive_targets.front()->kind->device_type` here?



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