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 2020/09/08 23:31:16 UTC

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6369: [Target] Target Tags, Composite Target and Unified Interface

comaniac commented on a change in pull request #6369:
URL: https://github.com/apache/incubator-tvm/pull/6369#discussion_r485220992



##########
File path: python/tvm/relay/backend/graph_runtime_codegen.py
##########
@@ -55,11 +56,11 @@ def _setup(self, mod, target):
         tgts = {}
         if isinstance(target, dict):

Review comment:
       Out of curiosity, what is this case for?

##########
File path: python/tvm/target/tag.py
##########
@@ -0,0 +1,70 @@
+# 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.
+"""Target tags"""
+from typing import Any, Dict
+from . import _ffi_api
+from .target import Target
+
+
+def list_tags() -> Dict[str, Target]:
+    """Returns a dict of tags, which maps each tag name to its corresponding target.
+
+    Returns
+    -------
+    tag_dict : Dict[str, Target]
+        The dict of tags mapping each tag name to to its corresponding target
+    """
+    return _ffi_api.TargetTagListTags()
+
+
+def register_tag(name: str, config: Dict[str, Any], override: bool = False) -> Target:
+    """Add a user-defined tag into the target tag registry.
+
+    Parameters
+    ----------
+    name: str
+        Name of the target, e.g. "nvidia/gtx1080ti"
+    config : Dict[str, Any]
+        The config dict used to create the target
+    override: bool
+        A boolean flag indicating if overriding existing tags are allowed
+
+    Returns
+    -------
+    target : Target
+        The target corresponding to the tag
+
+    Examples
+    --------
+    .. code-block:: python
+
+        register_tag("nvidia/gtx1080ti", config={
+            "kind": "cuda",
+            "arch": "sm_61",
+        })
+    """
+    return _ffi_api.TargetTagAddTag(name, config, override)
+
+
+# We do a round of tag listing on loading time to verify all tag information
+if hasattr(_ffi_api, "TargetTagAddTag"):
+    list_tags()
+
+    register_tag("nvidia/gtx1080ti", config={
+        "kind": "cuda",
+        "arch": "sm_61",
+    })

Review comment:
       Didn't understand this part. Could you elaborate?

##########
File path: src/target/target.cc
##########
@@ -57,119 +72,216 @@ static std::vector<String> DeduplicateKeys(const std::vector<String>& keys) {
   return new_keys;
 }
 
-static inline std::string RemovePrefixDashes(const std::string& s) {
-  size_t n_dashes = 0;
-  for (; n_dashes < s.length() && s[n_dashes] == '-'; ++n_dashes) {
+template <class TObj>
+static const TObj* ObjTypeCheck(const ObjectRef& obj, const std::string& expected_type) {
+  const TObj* ptr = obj.as<TObj>();
+  if (ptr == nullptr) {
+    std::ostringstream os;
+    os << ": Expects type \"" << expected_type << "\", but gets \"" << obj->GetTypeKey()

Review comment:
       nit: Could we get the type name from `TObj` directly so that we don't have to pass `expected_type`, which seems not quite necessary?

##########
File path: src/target/tag.cc
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 src/target/target_tag.cc
+ * \brief Target tag registry
+ */
+#include <tvm/runtime/registry.h>
+#include <tvm/target/tag.h>
+#include <tvm/target/target.h>
+
+#include "../node/attr_registry.h"
+
+namespace tvm {
+
+TVM_REGISTER_NODE_TYPE(TargetTagNode);
+
+TVM_REGISTER_GLOBAL("target.TargetTagListTags").set_body_typed(TargetTag::ListTags);
+TVM_REGISTER_GLOBAL("target.TargetTagAddTag").set_body_typed(TargetTag::AddTag);
+
+/**********  Registry-related code  **********/
+
+using TargetTagRegistry = AttrRegistry<TargetTagRegEntry, TargetTag>;
+
+TargetTagRegEntry& TargetTagRegEntry::RegisterOrGet(const String& target_tag_name) {
+  return TargetTagRegistry::Global()->RegisterOrGet(target_tag_name);
+}
+
+Optional<Target> TargetTag::Get(const String& target_tag_name) {
+  const TargetTagRegEntry* reg = TargetTagRegistry::Global()->Get(target_tag_name);
+  if (reg == nullptr) {
+    return NullOpt;
+  }
+  return Target(reg->tag_->config);
+}
+
+Map<String, Target> TargetTag::ListTags() {
+  Map<String, Target> result;
+  for (const String& tag : TargetTagRegistry::Global()->ListAllNames()) {
+    result.Set(tag, TargetTag::Get(tag).value());
+  }
+  return result;
+}
+
+Target TargetTag::AddTag(String name, Map<String, ObjectRef> config, bool override) {
+  TargetTagRegEntry& tag = TargetTagRegEntry::RegisterOrGet(name).set_name();
+  CHECK(override || tag.tag_->config.empty())
+      << "Tag \"" << name << "\" has been previously defined as: " << tag.tag_->config;
+  tag.set_config(config);
+  return Target(config);
+}
+
+/**********  Register Target tags  **********/

Review comment:
       Just confirm. So is that the place we suppose to let everyone register target tags? If so, we probably need to move the current tags in `python/tvm/target/target.py`, such as ARM CPU devices, to here.

##########
File path: src/target/target_kind.cc
##########
@@ -52,178 +57,238 @@ const AttrRegistryMapContainerMap<TargetKind>& TargetKind::GetAttrMapContainer(
   return TargetKindRegistry::Global()->GetAttrMap(attr_name);
 }
 
-const TargetKind& TargetKind::Get(const String& target_kind_name) {
+Optional<TargetKind> TargetKind::Get(const String& target_kind_name) {
   const TargetKindRegEntry* reg = TargetKindRegistry::Global()->Get(target_kind_name);
-  CHECK(reg != nullptr) << "ValueError: TargetKind \"" << target_kind_name
-                        << "\" is not registered";
+  if (reg == nullptr) {
+    return NullOpt;
+  }
   return reg->kind_;
 }
 
-// TODO(@junrushao1994): remove some redundant attributes
+/**********  Utility functions  **********/

Review comment:
       Can we move all utility functions in `target_kind.cc` and `target.cc` to `utils.cc`? It should be clearer and concise.

##########
File path: python/tvm/target/target.py
##########
@@ -126,7 +172,7 @@ def cuda(model='unknown', options=None):
         Additional options
     """
     opts = _merge_opts(['-model=%s' % model], options)
-    return _ffi_api.TargetCreate("cuda", *opts)
+    return Target(" ".join(["cuda"] + opts))

Review comment:
       Given that we attempt to use JSON as the majority of target representation, would that be better to improve syntactic sugar to something like the following?
   
   ```python
   def cuda(model: str = 'unknown', options: Optional[Union[str, Dict[str, Any]]] = None):
       if isinstance(options, str):
           options = parse_attr_from_str(options)
       options['model'] = model
       options['keys'] = ['cuda']
       return Target(options)
   ```

##########
File path: python/tvm/target/tag.py
##########
@@ -0,0 +1,70 @@
+# 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.
+"""Target tags"""
+from typing import Any, Dict
+from . import _ffi_api
+from .target import Target
+
+
+def list_tags() -> Dict[str, Target]:
+    """Returns a dict of tags, which maps each tag name to its corresponding target.
+
+    Returns
+    -------
+    tag_dict : Dict[str, Target]
+        The dict of tags mapping each tag name to to its corresponding target
+    """
+    return _ffi_api.TargetTagListTags()
+
+
+def register_tag(name: str, config: Dict[str, Any], override: bool = False) -> Target:

Review comment:
       Some opinions to this API:
   1. If `override` is `False`, an error will be threw out based on the C++ implementation. The description should be mentioned in the `override` docstring.
   2. It seems not necessary to return a target in this function. It makes more sense for this function to accept a `Target` directly; otherwise it would be better to ask users to explicitly call `Target(config)` if they need to use the target object after this registration.

##########
File path: python/tvm/target/target.py
##########
@@ -70,7 +116,7 @@ def current(allow_none=True):
         ------
         ValueError if current target is not set.
         """
-        return _ffi_api.GetCurrentTarget(allow_none)
+        return _ffi_api.TargetCurrent(allow_none)

Review comment:
       Would `CurrentTarget` more straightforward?

##########
File path: python/tvm/target/target.py
##########
@@ -37,22 +38,67 @@ class Target(Object):
 
     Note
     ----
-    Do not use class constructor, you can create target using the following functions
+    You can create target using the constructor or the following functions
 
-    - :py:func:`tvm.target.create` create target from string
     - :py:func:`tvm.target.arm_cpu` create arm_cpu target
     - :py:func:`tvm.target.cuda` create CUDA target
     - :py:func:`tvm.target.rocm` create ROCM target
     - :py:func:`tvm.target.mali` create Mali target
     - :py:func:`tvm.target.intel_graphics` create Intel Graphics target
     """
 
+    def __init__(self, tag_or_str_or_dict):
+        """Construct a TVM target object from
+        1) Raw target string
+        2) Target config dict
+        3) Target tag
+
+        Parameters
+        ----------
+        tag_or_str_or_dict : str or dict

Review comment:
       `dict` for numpy style type annotation, and `Dict[?]` for PEP8 style. Please be consistent at least in this PR. (ditto to all docstring types in this function.)

##########
File path: src/target/target.cc
##########
@@ -225,35 +325,52 @@ Optional<String> TargetNode::StringifyAttrsToRaw(const Map<String, ObjectRef>& a
   return JoinString(result, ' ');
 }
 
-Target Target::CreateTarget(const std::string& name, const std::vector<std::string>& options) {
-  TargetKind kind = TargetKind::Get(name);
-  ObjectPtr<TargetNode> target = make_object<TargetNode>();
-  target->kind = kind;
-  // tag is always empty
-  target->tag = "";
-  // parse attrs
-  target->attrs = target->ParseAttrsFromRaw(options);
-  String device_name = target->GetAttr<String>("device", "").value();
-  // set up keys
-  {
-    std::vector<String> keys;
-    // user provided keys
-    if (Optional<Array<String>> user_keys = target->GetAttr<Array<String>>("keys")) {
-      keys = std::vector<String>(user_keys.value().begin(), user_keys.value().end());
-      target->attrs.erase("keys");
-    }
-    // add `device_name`
-    if (!device_name.empty()) {
-      keys.push_back(device_name);
+const std::string& TargetNode::str() const {

Review comment:
       I recall that we are going to deprecate this function as well? If so it would be better to add a comment (but not a warning since it would be too annoying...)

##########
File path: python/tvm/target/target.py
##########
@@ -348,66 +393,26 @@ def create_llvm(llvm_args):
     llvm_str = create_llvm(llvm_args)
     args_list = target_str.split() + llvm_str.split()
 
-    return _ffi_api.TargetCreate('hexagon', *args_list)
+    return Target(" ".join(["hexagon"] + args_list))
 
 
 def create(target):
-    """Get a target given target string.
+    """Deprecated. Use the constructor of :py:mod:`tvm.target.Target` directly.
+    """
+    warnings.warn('tvm.target.create() is going to be deprecated. '
+                  'Please use the constructor of tvm.target.Target directly')

Review comment:
       ```suggestion
       warnings.warn('tvm.target.create() is being deprecated. '
                     'Please tvm.target.Target() instead')
   ```

##########
File path: include/tvm/target/tag.h
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 tvm/target/tag.h
+ * \brief Target tag registry
+ */
+#ifndef TVM_TARGET_TAG_H_
+#define TVM_TARGET_TAG_H_
+
+#include <tvm/node/attr_registry_map.h>
+#include <tvm/node/node.h>
+#include <tvm/target/target.h>
+
+#include <utility>
+
+namespace tvm {
+
+/*! \brief A target tag */
+class TargetTagNode : public Object {
+ public:
+  /*! \brief Name of the target */
+  String name;
+  /*! \brief Config map to generate the target */
+  Map<String, ObjectRef> config;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("config", &config);
+  }
+
+  static constexpr const char* _type_key = "TargetTag";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TargetTagNode, Object);
+
+ private:
+  /*! \brief Return the index stored in attr registry */
+  uint32_t AttrRegistryIndex() const { return index_; }
+  /*! \brief Return the name stored in attr registry */
+  String AttrRegistryName() const { return name; }
+  /*! \brief Index used for internal lookup of attribute registry */
+  uint32_t index_;
+
+  template <typename, typename>
+  friend class AttrRegistry;
+  template <typename>
+  friend class AttrRegistryMapContainerMap;
+  friend class TargetTagRegEntry;
+};
+
+/*!
+ * \brief Managed reference class to TargetTagNode
+ * \sa TargetTagNode
+ */
+class TargetTag : public ObjectRef {
+ public:
+  /*!
+   * \brief Retrieve the Target given it the name of target tag
+   * \param target_tag_name Name of the target tag
+   * \return The Target requested
+   */
+  TVM_DLL static Optional<Target> Get(const String& target_tag_name);
+  /*!
+   * \brief List all names of the existing target tags
+   * \return A dictionary that maps tag name to the concrete target it corresponds to
+   */
+  TVM_DLL static Map<String, Target> ListTags();
+  /*!
+   * \brief Add a tag into the registry
+   * \param name Name of the tag
+   * \param config The target config corresponding to the tag
+   * \param override Allow overriding existing tags
+   * \return Target created with the tag
+   */
+  TVM_DLL static Target AddTag(String name, Map<String, ObjectRef> config, bool override);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TargetTag, ObjectRef, TargetTagNode);
+
+ private:
+  /*! \brief Mutable access to the container class  */
+  TargetTagNode* operator->() { return static_cast<TargetTagNode*>(data_.get()); }
+  friend class TargetTagRegEntry;
+};
+
+class TargetTagRegEntry {
+ public:
+  /*!
+   * \brief Set the config dict corresponding to the target tag
+   * \param config The config dict for target creation
+   */
+  inline TargetTagRegEntry& set_config(Map<String, ObjectRef> config);
+  /*! \brief Set name of the TargetTag to be the same as registry if it is empty */
+  inline TargetTagRegEntry& set_name();
+  /*!
+   * \brief Register or get a new entry.
+   * \param target_tag_name The name of the TargetTag.
+   * \return the corresponding entry.
+   */
+  TVM_DLL static TargetTagRegEntry& RegisterOrGet(const String& target_tag_name);
+
+ private:
+  TargetTag tag_;
+  String name;

Review comment:
       * s/name/name_
   * Better to be clearer, like `tag_name_`.

##########
File path: include/tvm/target/tag.h
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 tvm/target/tag.h
+ * \brief Target tag registry
+ */
+#ifndef TVM_TARGET_TAG_H_
+#define TVM_TARGET_TAG_H_
+
+#include <tvm/node/attr_registry_map.h>
+#include <tvm/node/node.h>
+#include <tvm/target/target.h>
+
+#include <utility>
+
+namespace tvm {
+
+/*! \brief A target tag */
+class TargetTagNode : public Object {
+ public:
+  /*! \brief Name of the target */
+  String name;
+  /*! \brief Config map to generate the target */
+  Map<String, ObjectRef> config;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("config", &config);
+  }
+
+  static constexpr const char* _type_key = "TargetTag";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TargetTagNode, Object);
+
+ private:
+  /*! \brief Return the index stored in attr registry */
+  uint32_t AttrRegistryIndex() const { return index_; }
+  /*! \brief Return the name stored in attr registry */
+  String AttrRegistryName() const { return name; }
+  /*! \brief Index used for internal lookup of attribute registry */
+  uint32_t index_;
+
+  template <typename, typename>
+  friend class AttrRegistry;
+  template <typename>
+  friend class AttrRegistryMapContainerMap;
+  friend class TargetTagRegEntry;
+};
+
+/*!
+ * \brief Managed reference class to TargetTagNode
+ * \sa TargetTagNode
+ */
+class TargetTag : public ObjectRef {
+ public:
+  /*!
+   * \brief Retrieve the Target given it the name of target tag
+   * \param target_tag_name Name of the target tag
+   * \return The Target requested
+   */
+  TVM_DLL static Optional<Target> Get(const String& target_tag_name);
+  /*!
+   * \brief List all names of the existing target tags
+   * \return A dictionary that maps tag name to the concrete target it corresponds to
+   */
+  TVM_DLL static Map<String, Target> ListTags();
+  /*!
+   * \brief Add a tag into the registry
+   * \param name Name of the tag
+   * \param config The target config corresponding to the tag
+   * \param override Allow overriding existing tags
+   * \return Target created with the tag
+   */
+  TVM_DLL static Target AddTag(String name, Map<String, ObjectRef> config, bool override);

Review comment:
       These two APIs make me totally confused...I suppose `TargetTag` maintains the target config of one target tag, but why it has those two APIs that look like managing all the tags?
   
   In addition, I feel this class is not a "target tag". Target tag is more like a string IMO. This looks more like a container to keep a target config. In short, currently we have
   ```
   name -> TargetTag
   ```
   intuitively, we could have
   ```
   tag_name -> TargetConfig
   ```

##########
File path: src/target/target.cc
##########
@@ -288,151 +405,177 @@ Map<String, ObjectRef> TargetNode::Export() const {
   return result;
 }
 
-const std::string& TargetNode::str() const {
-  if (str_repr_.empty()) {
-    std::ostringstream os;
-    os << kind->name;
-    if (!this->keys.empty()) {
-      os << " -keys=";
-      bool is_first = true;
-      for (const String& s : keys) {
-        if (is_first) {
-          is_first = false;
-        } else {
-          os << ',';
-        }
-        os << s;
-      }
-    }
-    if (Optional<String> attrs_str = this->StringifyAttrsToRaw(attrs)) {
-      os << ' ' << attrs_str.value();
+/*! \brief Entry to hold the Target context stack. */
+struct TVMTargetThreadLocalEntry {
+  /*! \brief The current target context */
+  std::stack<Target> context_stack;
+};
+
+/*! \brief Thread local store to hold the Target context stack. */
+using TVMTargetThreadLocalStore = dmlc::ThreadLocalStore<TVMTargetThreadLocalEntry>;
+
+void Target::EnterWithScope() {
+  TVMTargetThreadLocalEntry* entry = TVMTargetThreadLocalStore::Get();
+  entry->context_stack.push(*this);
+}
+
+void Target::ExitWithScope() {
+  TVMTargetThreadLocalEntry* entry = TVMTargetThreadLocalStore::Get();
+  CHECK(!entry->context_stack.empty());
+  CHECK(entry->context_stack.top().same_as(*this));
+  entry->context_stack.pop();
+}
+
+Target Target::Current(bool allow_not_defined) {
+  TVMTargetThreadLocalEntry* entry = TVMTargetThreadLocalStore::Get();
+  if (entry->context_stack.size() > 0) {
+    return entry->context_stack.top();
+  }
+  CHECK(allow_not_defined)
+      << "Target context required. Please set it by constructing a TargetContext";
+
+  return Target();
+}
+
+/**********  Creation  **********/
+
+void TargetInternal::ConstructorDispatcher(TVMArgs args, TVMRetValue* rv) {
+  if (args.num_args == 1) {
+    const auto& arg = args[0];
+    if (arg.IsObjectRef<Target>()) {
+      *rv = Target(arg.AsObjectRef<Target>());
+    } else if (String::CanConvertFrom(arg)) {
+      *rv = Target(arg.operator String());
+    } else if (arg.IsObjectRef<Map<String, ObjectRef>>()) {
+      *rv = Target(arg.operator Map<String, ObjectRef>());
+    } else if (arg.type_code() == kTVMObjectHandle) {
+      ObjectRef obj = arg;
+      LOG(FATAL) << "TypeError: Cannot create target with type: " << obj->GetTypeKey();
+    } else {
+      LOG(FATAL) << "TypeError: Cannot create target with type: "
+                 << runtime::ArgTypeCode2Str(arg.type_code());
     }
-    str_repr_ = os.str();
+    return;
   }
-  return str_repr_;
+  LOG(FATAL) << "ValueError: Invalid number of arguments. Expect 1, but gets: " << args.num_args;
 }
 
-bool StartsWith(const std::string& str, const std::string& pattern) {
-  return str.compare(0, pattern.length(), pattern) == 0;
+ObjectPtr<Object> TargetInternal::FromString(const String& tag_or_config_or_target_str) {
+  if (Optional<Target> target = TargetTag::Get(tag_or_config_or_target_str)) {
+    Target value = target.value();
+    return runtime::ObjectInternal::MoveObjectPtr(&value);
+  }
+  if (!tag_or_config_or_target_str.empty() && tag_or_config_or_target_str.data()[0] == '{') {
+    return TargetInternal::FromConfigString(tag_or_config_or_target_str);
+  }
+  return TargetInternal::FromRawString(tag_or_config_or_target_str);

Review comment:
       Would that be better to use try-FromConfigString-catch-FromRawString? It seems to me that `tag_or_config_or_target_str.data()[0] == '{'` is not very promising. What if the string is `<spaces> { ... }`?

##########
File path: src/target/target_kind.cc
##########
@@ -52,178 +57,238 @@ const AttrRegistryMapContainerMap<TargetKind>& TargetKind::GetAttrMapContainer(
   return TargetKindRegistry::Global()->GetAttrMap(attr_name);
 }
 
-const TargetKind& TargetKind::Get(const String& target_kind_name) {
+Optional<TargetKind> TargetKind::Get(const String& target_kind_name) {
   const TargetKindRegEntry* reg = TargetKindRegistry::Global()->Get(target_kind_name);
-  CHECK(reg != nullptr) << "ValueError: TargetKind \"" << target_kind_name
-                        << "\" is not registered";
+  if (reg == nullptr) {
+    return NullOpt;
+  }
   return reg->kind_;
 }
 
-// TODO(@junrushao1994): remove some redundant attributes
+/**********  Utility functions  **********/
 
-TVM_REGISTER_TARGET_KIND("llvm")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
-    .add_attr_option<Bool>("system-lib")
-    .add_attr_option<String>("runtime")
-    .add_attr_option<String>("mcpu")
+/*!
+ * \brief Extract a number from the string with the given prefix.
+ * For example, when `str` is "sm_20" and `prefix` is "sm_".
+ * This function first checks if `str` starts with `prefix`,
+ * then return the integer 20 after the `prefix`
+ * \param str The string to be extracted
+ * \param prefix The prefix to be checked
+ * \return An integer, the extracted number. -1 if the check fails
+ */
+static int ExtractIntWithPrefix(const std::string& str, const std::string& prefix) {
+  if (str.substr(0, prefix.size()) != prefix) {
+    return -1;
+  }
+  int result = 0;
+  for (size_t i = prefix.size(); i < str.size(); ++i) {
+    char c = str[i];
+    if (!isdigit(c)) {
+      return -1;
+    }
+    result = result * 10 + c - '0';
+  }
+  return result;
+}
+
+/*!
+ * \brief Using TVM DeviceAPI to detect the device flag
+ * \param device The device to be detected
+ * \param flag The device flag to be detected
+ * \param val The detected value
+ * \return A boolean indicating if detection succeeds
+ */
+static bool DetectDeviceFlag(TVMContext device, runtime::DeviceAttrKind flag, TVMRetValue* val) {
+  using runtime::DeviceAPI;
+  DeviceAPI* api = DeviceAPI::Get(device, true);
+  // Check if compiled with the corresponding device api
+  if (api == nullptr) {
+    return false;
+  }
+  // Check if the device exists
+  api->GetAttr(device, runtime::kExist, val);
+  int exists = *val;
+  if (!exists) {
+    return false;
+  }
+  // Get the arch of the device
+  DeviceAPI::Get(device)->GetAttr(device, flag, val);
+  return true;
+}
+
+void CheckOrSetAttr(Map<String, ObjectRef>* attrs, const String& name, const String& value) {
+  auto iter = attrs->find(name);
+  if (iter == attrs->end()) {
+    attrs->Set(name, value);
+  } else {
+    const auto* str = (*iter).second.as<StringObj>();
+    CHECK(str != nullptr && GetRef<String>(str) == value)
+        << "ValueError: Expects \"" << name << "\" to be \"" << value
+        << "\", but gets: " << (*iter).second;
+  }
+}
+
+/**********  Target kind attribute updaters  **********/
+
+/*!
+ * \brief Update the attributes in the LLVM NVPTX target.
+ * \param attrs The original attributes
+ * \return The updated attributes
+ */
+Map<String, ObjectRef> UpdateNVPTXAttrs(Map<String, ObjectRef> attrs) {
+  CheckOrSetAttr(&attrs, "mtriple", "nvptx64-nvidia-cuda");
+  // Update -mcpu=sm_xx
+  int arch;
+  if (attrs.count("mcpu")) {
+    // If -mcpu has been specified, validate the correctness
+    String mcpu = Downcast<String>(attrs.at("mcpu"));
+    arch = ExtractIntWithPrefix(mcpu, "sm_");
+    CHECK(arch != -1) << "ValueError: NVPTX target gets an invalid CUDA arch: -mcpu=" << mcpu;
+  } else {
+    // Use the compute version of the first CUDA GPU instead
+    TVMRetValue version;
+    if (!DetectDeviceFlag({kDLGPU, 0}, runtime::kComputeVersion, &version)) {
+      LOG(WARNING) << "Unable to detect CUDA version, default to \"-mcpu=sm_20\" instead";
+      arch = 20;
+    } else {
+      arch = std::stod(version.operator std::string()) * 10 + 0.1;
+    }
+    attrs.Set("mcpu", String("sm_") + std::to_string(arch));
+  }
+  return attrs;
+}
+
+/*!
+ * \brief Update the attributes in the LLVM ROCm target.
+ * \param attrs The original attributes
+ * \return The updated attributes
+ */
+Map<String, ObjectRef> UpdateROCmAttrs(Map<String, ObjectRef> attrs) {
+  CheckOrSetAttr(&attrs, "mtriple", "amdgcn-amd-amdhsa-hcc");
+  // Update -mcpu=gfx
+  int arch;
+  if (attrs.count("mcpu")) {
+    String mcpu = Downcast<String>(attrs.at("mcpu"));
+    arch = ExtractIntWithPrefix(mcpu, "gfx");
+    CHECK(arch != -1) << "ValueError: ROCm target gets an invalid GFX version: -mcpu=" << mcpu;
+  } else {
+    TVMRetValue version;
+    if (!DetectDeviceFlag({kDLROCM, 0}, runtime::kApiVersion, &version)) {
+      LOG(WARNING) << "Unable to detect ROCm version, default to \"-mcpu=gfx305\" instead";
+      arch = 305;
+    } else {
+      arch = version.operator int();
+    }
+    attrs.Set("mcpu", String("gfx") + std::to_string(arch));
+  }
+  // Update -mattr before ROCm 3.5:
+  //   Before ROCm 3.5 we needed code object v2, starting
+  //   with 3.5 we need v3 (this argument disables v3)
+  if (arch < 305) {
+    Array<String> mattr;
+    if (attrs.count("mattr")) {
+      mattr = Downcast<Array<String>>(attrs.at("mattr"));
+    }
+    mattr.push_back("-code-object-v3");
+    attrs.Set("mattr", mattr);
+  }
+  return attrs;
+}
+
+/**********  Register Target kinds and attributes  **********/
+
+TVM_REGISTER_TARGET_KIND("llvm", kDLCPU)
     .add_attr_option<Array<String>>("mattr")
+    .add_attr_option<String>("mcpu")
     .add_attr_option<String>("mtriple")
     .add_attr_option<String>("mfloat-abi")
-    .set_default_keys({"cpu"})
-    .set_device_type(kDLCPU);
-
-TVM_REGISTER_TARGET_KIND("c")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<String>("runtime")
-    .set_default_keys({"cpu"})
-    .set_device_type(kDLCPU);
-
-TVM_REGISTER_TARGET_KIND("cuda")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"cpu"});
+
+TVM_REGISTER_TARGET_KIND("c", kDLCPU)
     .add_attr_option<Bool>("system-lib")
-    .add_attr_option<Integer>("max_num_threads", Integer(1024))
-    .add_attr_option<Integer>("thread_warp_size", Integer(32))
-    .add_attr_option<String>("mcpu")
-    .set_default_keys({"cuda", "gpu"})
-    .set_device_type(kDLGPU);
+    .add_attr_option<String>("runtime")
+    .set_default_keys({"cpu"});
 
-TVM_REGISTER_TARGET_KIND("nvptx")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+TVM_REGISTER_TARGET_KIND("cuda", kDLGPU)
+    .add_attr_option<String>("mcpu")
+    .add_attr_option<String>("arch")
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(1024))
     .add_attr_option<Integer>("thread_warp_size", Integer(32))
+    .set_default_keys({"cuda", "gpu"});
+
+TVM_REGISTER_TARGET_KIND("nvptx", kDLGPU)
     .add_attr_option<String>("mcpu")
     .add_attr_option<String>("mtriple")
+    .add_attr_option<Bool>("system-lib")
+    .add_attr_option<Integer>("max_num_threads", Integer(1024))
+    .add_attr_option<Integer>("thread_warp_size", Integer(32))
     .set_default_keys({"cuda", "gpu"})
-    .set_device_type(kDLGPU);
+    .set_attrs_preprocessor(UpdateNVPTXAttrs);
 
-TVM_REGISTER_TARGET_KIND("rocm")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+TVM_REGISTER_TARGET_KIND("rocm", kDLROCM)
+    .add_attr_option<String>("mcpu")
+    .add_attr_option<String>("mtriple")
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
     .add_attr_option<Integer>("thread_warp_size", Integer(64))
     .set_default_keys({"rocm", "gpu"})
-    .set_device_type(kDLROCM);
+    .set_attrs_preprocessor(UpdateROCmAttrs);
 
-TVM_REGISTER_TARGET_KIND("opencl")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+TVM_REGISTER_TARGET_KIND("opencl", kDLOpenCL)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
     .add_attr_option<Integer>("thread_warp_size")
-    .set_default_keys({"opencl", "gpu"})
-    .set_device_type(kDLOpenCL);
-
-TVM_REGISTER_TARGET_KIND("metal")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"opencl", "gpu"});
+
+TVM_REGISTER_TARGET_KIND("metal", kDLMetal)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
-    .set_default_keys({"metal", "gpu"})
-    .set_device_type(kDLMetal);
-
-TVM_REGISTER_TARGET_KIND("vulkan")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"metal", "gpu"});
+
+TVM_REGISTER_TARGET_KIND("vulkan", kDLVulkan)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
-    .set_default_keys({"vulkan", "gpu"})
-    .set_device_type(kDLVulkan);
-
-TVM_REGISTER_TARGET_KIND("webgpu")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"vulkan", "gpu"});
+
+TVM_REGISTER_TARGET_KIND("webgpu", kDLWebGPU)
     .add_attr_option<Bool>("system-lib")
     .add_attr_option<Integer>("max_num_threads", Integer(256))
-    .set_default_keys({"webgpu", "gpu"})
-    .set_device_type(kDLWebGPU);
-
-TVM_REGISTER_TARGET_KIND("sdaccel")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"webgpu", "gpu"});
+
+TVM_REGISTER_TARGET_KIND("sdaccel", kDLOpenCL)
     .add_attr_option<Bool>("system-lib")
-    .set_default_keys({"sdaccel", "hls"})
-    .set_device_type(kDLOpenCL);
-
-TVM_REGISTER_TARGET_KIND("aocl")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"sdaccel", "hls"});
+
+TVM_REGISTER_TARGET_KIND("aocl", kDLAOCL)
     .add_attr_option<Bool>("system-lib")
-    .set_default_keys({"aocl", "hls"})
-    .set_device_type(kDLAOCL);
-
-TVM_REGISTER_TARGET_KIND("aocl_sw_emu")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
+    .set_default_keys({"aocl", "hls"});
+
+TVM_REGISTER_TARGET_KIND("aocl_sw_emu", kDLAOCL)
     .add_attr_option<Bool>("system-lib")
-    .set_default_keys({"aocl", "hls"})
-    .set_device_type(kDLAOCL);
-
-TVM_REGISTER_TARGET_KIND("hexagon")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
-    .add_attr_option<String>("mcpu")
+    .set_default_keys({"aocl", "hls"});
+
+TVM_REGISTER_TARGET_KIND("hexagon", kDLHexagon)
     .add_attr_option<Array<String>>("mattr")
+    .add_attr_option<String>("mcpu")
     .add_attr_option<String>("mtriple")
-    .add_attr_option<Array<String>>("llvm-options")
     .add_attr_option<Bool>("system-lib")
-    .set_default_keys({"hexagon"})
-    .set_device_type(kDLHexagon);
-
-TVM_REGISTER_TARGET_KIND("stackvm")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
-    .add_attr_option<Bool>("system-lib")
-    .set_device_type(kDLCPU);
+    .add_attr_option<Array<String>>("llvm-options")
+    .set_default_keys({"hexagon"});
 
-TVM_REGISTER_TARGET_KIND("ext_dev")
-    .add_attr_option<Array<String>>("keys")
-    .add_attr_option<Array<String>>("libs")
-    .add_attr_option<String>("device")
-    .add_attr_option<String>("model")
-    .add_attr_option<Bool>("system-lib")
-    .set_device_type(kDLExtDev);
+TVM_REGISTER_TARGET_KIND("stackvm", kDLCPU)  // line break

Review comment:
       Out of curiosity, what does `// line break` do?




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