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/02/06 00:01:08 UTC

[GitHub] [tvm] comaniac commented on a change in pull request #7304: [TVMC] Add custom codegen (BYOC) passes for compilation and tuning

comaniac commented on a change in pull request #7304:
URL: https://github.com/apache/tvm/pull/7304#discussion_r571280625



##########
File path: python/tvm/driver/tvmc/byoc.py
##########
@@ -0,0 +1,191 @@
+# 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.
+"""
+Provides support to Bring Your Own Codegen (BYOC) on TVMC.

Review comment:
       BYOC is not the previous term from the TVMC's point of view. Users don't need to know what BYOC is, but they only know they are going to run the model on multiple platforms. As the post in the forum illustrates, users only provide multiple targest in order. As a result, the precise name of this feature in TVMC would be something like "composite target support".

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -91,18 +272,37 @@ def target_from_cli(target):
     -------
     tvm.target.Target
         an instance of target device information
+    codegens : list of dict
+        This list preserves the order in which codegens were
+        provided via command line. Each Dict contains three keys:
+        'kind', containing the name of the codegen; 'opts' containing
+        a key-value for all options passed via CLI; 'raw',
+        containing the plain string for this codegen
     """
+    extra_codegens = []
 
     if os.path.exists(target):
         with open(target) as target_file:
-            logger.info("using target input from file: %s", target)
+            logger.info("target input is a path: %s", target)
             target = "".join(target_file.readlines())
+    elif is_inline_json(target):
+        logger.info("target input is inline JSON: %s", target)
+    else:
+        logger.info("target input is plain text: %s", target)
+        try:
+            parsed_targets = parse_target(target)
+        except ValueError as ex:
+            raise TVMCException(f"Error parsing target string '{target}'.\nThe error was: {ex}")
+
+        validate_targets(parsed_targets)
+        target = parsed_targets[-1]["raw"]
+        extra_codegens = parsed_targets[:-1] if len(parsed_targets) > 1 else []
 
     # TODO(@leandron) We don't have an API to collect a list of supported
     #       targets yet
     logger.debug("creating target from input: %s", target)
 
-    return tvm.target.Target(target)
+    return tvm.target.Target(target), extra_codegens

Review comment:
       So only plain text target supports composite target? Seems doesn't make sense to me.

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -31,6 +33,26 @@
 # pylint: disable=invalid-name
 logger = logging.getLogger("TVMC")
 
+TVM_TARGETS = [

Review comment:
       This list should not be in TVMC. TVM target should provide an API to generate this list.
   
   cc @junrushao1994 

##########
File path: python/tvm/driver/tvmc/byoc.py
##########
@@ -0,0 +1,191 @@
+# 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.
+"""
+Provides support to Bring Your Own Codegen (BYOC) on TVMC.
+"""
+import logging
+
+from abc import ABC
+from abc import abstractmethod
+
+import tvm
+
+from tvm import relay
+
+from tvm.relay.op.contrib import get_pattern_table
+
+from .common import TVMCException
+
+
+# pylint: disable=invalid-name
+logger = logging.getLogger("TVMC")
+
+# Global dictionary to map existing custom codegens
+# with the names used to use them
+REGISTERED_CODEGEN = {}
+
+
+def register_codegen(kind):
+    """
+    Utility function to register a BYOC class for TVMC.
+
+    Classes decorated with `tvm.driver.tvmc.target.register_codegen` will
+    be added to the codegens dictionary.
+
+    Example
+    -------
+
+        @register_codegen(kind="samplebyoc")
+        class MyCustomTarget(TVMCCodegen):
+            ...
+    """
+
+    def codegen_decorator(cls):
+        cls.kind = kind
+
+        assert kind not in REGISTERED_CODEGEN, "there is already a codegen '%s': %s" % (
+            kind,
+            REGISTERED_CODEGEN[kind],
+        )
+        assert issubclass(cls, TVMCCodegen), "%s is expected to be a subclass of TVMCCodegen" % cls
+
+        REGISTERED_CODEGEN[kind] = cls
+        return cls
+
+    return codegen_decorator
+
+
+def get_codegen_kinds():
+    """Return a list of all registered codegens.
+
+    Returns
+    --------
+    list of str
+        all registered codegens
+    """
+    return REGISTERED_CODEGEN.keys()
+
+
+def get_codegen_by_kind(kind):
+    """Return a custom codegen by kind.
+
+    Returns
+    --------
+    TVMCCodegen
+        The requested codegen or None in case it is
+        not valid
+    """
+    try:
+        return REGISTERED_CODEGEN[kind]
+    except KeyError:
+        raise TVMCException("Target %s is not defined." % kind)
+
+
+class TVMCCodegen(ABC):
+    """Abstract class for command line driver BYOC definition.
+
+    Provide a unified way to create a codegen adapter with a set of
+    callback functions to be used within TVMC.
+
+    """
+
+    @staticmethod
+    @abstractmethod
+    def get_config_key_name():
+        """Return the name of the dictionary key to be used
+        at compile time (i.e. relay.build) as part of the
+        "config" argument.
+
+        Returns
+        -------
+        str
+            Name of the Codegen config dictionary key
+            to match with the expected name in the relay
+            implementation of this Codegen.
+        """
+
+    @staticmethod
+    @abstractmethod
+    def run_custom_passes(mod, params):
+        """Apply a set of transformations to the module
+        before compilation happens.
+
+        Parameters
+        ----------
+        mod : tvm.relay.Module
+            The relay module to convert.
+        params : dict
+            The parameters (weights) for the TVM module.
+
+        Returns
+        -------
+        mod: tvm.relay.Module
+            The converted module.
+        """
+
+
+@register_codegen(kind="ethos-n77")
+class TVMCEthosNCodegen(TVMCCodegen):

Review comment:
       Please also add others, not just Ethos-N.

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -76,6 +98,165 @@ def convert_graph_layout(mod, desired_layout):
             )
 
 
+def validate_targets(parse_targets):
+    """
+    Apply a series of validations in the targets provided via CLI.
+    """
+    targets = [t["kind"] for t in parse_targets]
+
+    if len(targets) > len(set(targets)):
+        raise TVMCException("Duplicate target definitions are not allowed")
+
+    if targets[-1] not in TVM_TARGETS:
+        raise TVMCException(f"The last target needs to be a TVM target. Choices: {TVM_TARGETS}")
+
+    tvm_targets = [t for t in targets if t in TVM_TARGETS]
+    tvm_targets_count = len(tvm_targets)
+    if tvm_targets_count > 1:
+        verbose_tvm_targets = ", ".join(tvm_targets)
+        raise TVMCException(
+            f"Only one of the following targets can be used at a time. "
+            "Found {tvm_targets_count}: {verbose_tvm_targets}."

Review comment:
       ```suggestion
               "Found {verbose_tvm_targets}."
   ```

##########
File path: python/tvm/driver/tvmc/byoc.py
##########
@@ -0,0 +1,191 @@
+# 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.
+"""
+Provides support to Bring Your Own Codegen (BYOC) on TVMC.
+"""
+import logging
+
+from abc import ABC
+from abc import abstractmethod
+
+import tvm
+
+from tvm import relay
+
+from tvm.relay.op.contrib import get_pattern_table
+
+from .common import TVMCException
+
+
+# pylint: disable=invalid-name
+logger = logging.getLogger("TVMC")
+
+# Global dictionary to map existing custom codegens
+# with the names used to use them
+REGISTERED_CODEGEN = {}
+
+
+def register_codegen(kind):

Review comment:
       - Missing `Parameters` in docstring.
   - Why use "kind" here? From your use case it looks like just "name" or "target".

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -76,6 +98,165 @@ def convert_graph_layout(mod, desired_layout):
             )
 
 
+def validate_targets(parse_targets):
+    """
+    Apply a series of validations in the targets provided via CLI.
+    """
+    targets = [t["kind"] for t in parse_targets]
+
+    if len(targets) > len(set(targets)):
+        raise TVMCException("Duplicate target definitions are not allowed")
+
+    if targets[-1] not in TVM_TARGETS:
+        raise TVMCException(f"The last target needs to be a TVM target. Choices: {TVM_TARGETS}")
+
+    tvm_targets = [t for t in targets if t in TVM_TARGETS]
+    tvm_targets_count = len(tvm_targets)
+    if tvm_targets_count > 1:

Review comment:
       ```suggestion
       if len(tvm_targets) > 1:
   ```

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -91,18 +272,37 @@ def target_from_cli(target):
     -------
     tvm.target.Target
         an instance of target device information
+    codegens : list of dict

Review comment:
       Naming again. Users don't know what is codegen. All they need to know is they want their models to run on multiple devices.

##########
File path: python/tvm/driver/tvmc/byoc.py
##########
@@ -0,0 +1,191 @@
+# 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.
+"""
+Provides support to Bring Your Own Codegen (BYOC) on TVMC.
+"""
+import logging
+
+from abc import ABC
+from abc import abstractmethod
+
+import tvm
+
+from tvm import relay
+
+from tvm.relay.op.contrib import get_pattern_table
+
+from .common import TVMCException
+
+
+# pylint: disable=invalid-name
+logger = logging.getLogger("TVMC")
+
+# Global dictionary to map existing custom codegens
+# with the names used to use them
+REGISTERED_CODEGEN = {}
+
+
+def register_codegen(kind):
+    """
+    Utility function to register a BYOC class for TVMC.
+
+    Classes decorated with `tvm.driver.tvmc.target.register_codegen` will
+    be added to the codegens dictionary.
+
+    Example
+    -------
+
+        @register_codegen(kind="samplebyoc")
+        class MyCustomTarget(TVMCCodegen):
+            ...
+    """
+
+    def codegen_decorator(cls):
+        cls.kind = kind
+
+        assert kind not in REGISTERED_CODEGEN, "there is already a codegen '%s': %s" % (
+            kind,
+            REGISTERED_CODEGEN[kind],
+        )
+        assert issubclass(cls, TVMCCodegen), "%s is expected to be a subclass of TVMCCodegen" % cls
+
+        REGISTERED_CODEGEN[kind] = cls
+        return cls
+
+    return codegen_decorator
+
+
+def get_codegen_kinds():
+    """Return a list of all registered codegens.
+
+    Returns
+    --------
+    list of str
+        all registered codegens
+    """
+    return REGISTERED_CODEGEN.keys()

Review comment:
       Note that the type of `keys()` is not List.
   ```suggestion
       return list(REGISTERED_CODEGEN.keys())
   ```

##########
File path: python/tvm/driver/tvmc/byoc.py
##########
@@ -0,0 +1,191 @@
+# 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.
+"""
+Provides support to Bring Your Own Codegen (BYOC) on TVMC.
+"""
+import logging
+
+from abc import ABC
+from abc import abstractmethod
+
+import tvm
+
+from tvm import relay
+
+from tvm.relay.op.contrib import get_pattern_table
+
+from .common import TVMCException
+
+
+# pylint: disable=invalid-name
+logger = logging.getLogger("TVMC")
+
+# Global dictionary to map existing custom codegens
+# with the names used to use them
+REGISTERED_CODEGEN = {}
+
+
+def register_codegen(kind):
+    """
+    Utility function to register a BYOC class for TVMC.
+
+    Classes decorated with `tvm.driver.tvmc.target.register_codegen` will
+    be added to the codegens dictionary.
+
+    Example
+    -------
+
+        @register_codegen(kind="samplebyoc")
+        class MyCustomTarget(TVMCCodegen):
+            ...
+    """
+
+    def codegen_decorator(cls):
+        cls.kind = kind
+
+        assert kind not in REGISTERED_CODEGEN, "there is already a codegen '%s': %s" % (
+            kind,
+            REGISTERED_CODEGEN[kind],
+        )
+        assert issubclass(cls, TVMCCodegen), "%s is expected to be a subclass of TVMCCodegen" % cls
+
+        REGISTERED_CODEGEN[kind] = cls
+        return cls
+
+    return codegen_decorator
+
+
+def get_codegen_kinds():
+    """Return a list of all registered codegens.
+
+    Returns
+    --------
+    list of str
+        all registered codegens
+    """
+    return REGISTERED_CODEGEN.keys()
+
+
+def get_codegen_by_kind(kind):
+    """Return a custom codegen by kind.
+
+    Returns
+    --------
+    TVMCCodegen
+        The requested codegen or None in case it is
+        not valid
+    """
+    try:
+        return REGISTERED_CODEGEN[kind]
+    except KeyError:
+        raise TVMCException("Target %s is not defined." % kind)
+
+
+class TVMCCodegen(ABC):

Review comment:
       From your use case I think it is fine to simplify this class. The only things you want are 1) the config key name, and 2) the pass pipeline. Meanwhile, the pass pipeline should be defined some where else so we should not define it in TVMC. For example: ACL (https://github.com/apache/tvm/blob/main/python/tvm/relay/op/contrib/arm_compute_lib.py#L46) and TensorRT (https://github.com/apache/tvm/blob/main/python/tvm/relay/op/contrib/tensorrt.py#L81). 
   
   Accordingly, we only need one line to register them, such as 
   ```
   REGISTERED_CODEGEN[name] = (<config-name>, <pass pipeline function pointer>)
   ```
   
   so decotartor and this class can all be removed.

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -76,6 +98,165 @@ def convert_graph_layout(mod, desired_layout):
             )
 
 
+def validate_targets(parse_targets):
+    """
+    Apply a series of validations in the targets provided via CLI.
+    """
+    targets = [t["kind"] for t in parse_targets]
+
+    if len(targets) > len(set(targets)):
+        raise TVMCException("Duplicate target definitions are not allowed")
+
+    if targets[-1] not in TVM_TARGETS:
+        raise TVMCException(f"The last target needs to be a TVM target. Choices: {TVM_TARGETS}")
+
+    tvm_targets = [t for t in targets if t in TVM_TARGETS]
+    tvm_targets_count = len(tvm_targets)
+    if tvm_targets_count > 1:
+        verbose_tvm_targets = ", ".join(tvm_targets)
+        raise TVMCException(
+            f"Only one of the following targets can be used at a time. "
+            "Found {tvm_targets_count}: {verbose_tvm_targets}."
+        )
+
+
+def tokenize_target(target):
+    """
+    Extract a list of tokens from a target specification text.
+
+    It covers some corner-cases that are not covered by the built-in
+    module 'shlex', such as the use of "+" as a punctuation character.
+
+
+    Example
+    -------
+
+    For the input `foo -op1=v1 -op2="v ,2", bar -op3=v-4` we
+    should obtain:
+
+        ["foo", "-op1=v1", "-op2="v ,2"", ",", "bar", "-op3=v-4"]
+
+    Parameters
+    ----------
+    target : str
+        Target options sent via CLI arguments
+
+    Returns
+    -------
+    list of str
+        a list of parsed tokens extracted from the target string
+    """
+
+    target_pattern = (
+        r"(\-{0,2}[\w\-]+\=?"
+        r"(?:[\w\+\-]+(?:,[\w\+\-])*|[\'][\w\+\-,\s]+[\']|[\"][\w\+\-,\s]+[\"])*|,)"
+    )
+
+    return re.findall(target_pattern, target)
+
+
+def parse_target(target):
+    """
+    Parse a plain string of targets provided via a command-line
+    argument.
+
+    To send more than one codegen, a comma-separated list
+    is expected. Options start with -<option_name>=<value>.
+
+    We use python standard library 'shlex' to parse the argument in
+    a POSIX compatible way, so that if options are defined as
+    strings with spaces or commas, for example, this is considered
+    and parsed accordingly.
+
+
+    Example
+    -------
+
+    For the input `--target="foo -op1=v1 -op2="v ,2", bar -op3=v-4"` we
+    should obtain:
+
+      [
+        {
+            kind: "foo",
+            opts: {"op1":"v1", "op2":"v ,2"},
+            raw: 'foo -op1=v1 -op2="v ,2"'
+        },
+        {
+            kind: "bar",
+            opts: {"op3":"v-4"},
+            raw: 'bar -op3=v-4'
+        }
+      ]
+
+    Parameters
+    ----------
+    target : str
+        Target options sent via CLI arguments
+
+    Returns
+    -------
+    codegens : list of dict
+        This list preserves the order in which codegens were
+        provided via command line. Each Dict contains three keys:
+        'kind', containing the name of the codegen; 'opts' containing
+        a key-value for all options passed via CLI; 'raw',
+        containing the plain string for this codegen
+    """
+    codegens = []
+
+    parsed_tokens = tokenize_target(target)
+
+    split_codegens = []
+    current_codegen = []
+    split_codegens.append(current_codegen)
+    for token in parsed_tokens:
+        # every time there is a comma separating
+        # two codegen definitions, prepare for
+        # a new codegen
+        if token == ",":
+            current_codegen = []
+            split_codegens.append(current_codegen)
+        else:
+            # collect a new token for the current
+            # codegen being parsed
+            current_codegen.append(token)
+
+    # at this point we have a list of lists,
+    # each item on the first list is a codegen definition
+    # in the comma-separated values
+    for codegen_def in split_codegens:
+        # the first is expected to be the name
+        name = codegen_def[0]
+        raw_target = " ".join(codegen_def)
+        all_opts = codegen_def[1:] if len(codegen_def) > 1 else []
+        opts = {}
+        for opt in all_opts:
+            try:
+                # deal with -- prefixed flags
+                if opt.startswith("--"):
+                    opt_name = opt[2:]
+                    opt_value = True
+                else:
+                    opt = opt[1:] if opt.startswith("-") else opt
+                    opt_name, opt_value = opt.split("=", maxsplit=1)
+            except ValueError:
+                raise ValueError(f"Error when parsing '{opt}'")
+
+            opts[opt_name] = opt_value
+
+        codegens.append({"kind": name, "opts": opts, "raw": raw_target})
+
+    return codegens
+
+
+def is_inline_json(target):
+    try:
+        json.loads(target)
+        return True
+    except json.decoder.JSONDecodeError:
+        return False

Review comment:
       This function is only used by `target_from_cli` so it is more proper to put it inside.

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -91,18 +272,37 @@ def target_from_cli(target):
     -------
     tvm.target.Target
         an instance of target device information
+    codegens : list of dict
+        This list preserves the order in which codegens were
+        provided via command line. Each Dict contains three keys:
+        'kind', containing the name of the codegen; 'opts' containing
+        a key-value for all options passed via CLI; 'raw',
+        containing the plain string for this codegen
     """
+    extra_codegens = []
 
     if os.path.exists(target):
         with open(target) as target_file:
-            logger.info("using target input from file: %s", target)
+            logger.info("target input is a path: %s", target)
             target = "".join(target_file.readlines())
+    elif is_inline_json(target):
+        logger.info("target input is inline JSON: %s", target)

Review comment:
       1. You didn't assign target in this branch.
   2. Change the logging level to debug.

##########
File path: python/tvm/driver/tvmc/byoc.py
##########
@@ -0,0 +1,191 @@
+# 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.
+"""
+Provides support to Bring Your Own Codegen (BYOC) on TVMC.
+"""
+import logging
+
+from abc import ABC
+from abc import abstractmethod
+
+import tvm
+
+from tvm import relay
+
+from tvm.relay.op.contrib import get_pattern_table
+
+from .common import TVMCException
+
+
+# pylint: disable=invalid-name
+logger = logging.getLogger("TVMC")
+
+# Global dictionary to map existing custom codegens
+# with the names used to use them
+REGISTERED_CODEGEN = {}
+
+
+def register_codegen(kind):
+    """
+    Utility function to register a BYOC class for TVMC.
+
+    Classes decorated with `tvm.driver.tvmc.target.register_codegen` will
+    be added to the codegens dictionary.
+
+    Example
+    -------
+
+        @register_codegen(kind="samplebyoc")
+        class MyCustomTarget(TVMCCodegen):
+            ...
+    """
+
+    def codegen_decorator(cls):
+        cls.kind = kind
+
+        assert kind not in REGISTERED_CODEGEN, "there is already a codegen '%s': %s" % (
+            kind,
+            REGISTERED_CODEGEN[kind],
+        )
+        assert issubclass(cls, TVMCCodegen), "%s is expected to be a subclass of TVMCCodegen" % cls
+
+        REGISTERED_CODEGEN[kind] = cls
+        return cls
+
+    return codegen_decorator
+
+
+def get_codegen_kinds():
+    """Return a list of all registered codegens.
+
+    Returns
+    --------
+    list of str
+        all registered codegens
+    """
+    return REGISTERED_CODEGEN.keys()
+
+
+def get_codegen_by_kind(kind):
+    """Return a custom codegen by kind.

Review comment:
       Avoid using "custom" codegen. The concept is that once the codegen has been merged to TVM, it is no longer a "custom" codegen. It is also inconsistent to the "TVMCodegen" that needs to be registered. We need a better naming through this PR in general.

##########
File path: python/tvm/driver/tvmc/common.py
##########
@@ -76,6 +98,165 @@ def convert_graph_layout(mod, desired_layout):
             )
 
 
+def validate_targets(parse_targets):
+    """
+    Apply a series of validations in the targets provided via CLI.
+    """
+    targets = [t["kind"] for t in parse_targets]
+
+    if len(targets) > len(set(targets)):
+        raise TVMCException("Duplicate target definitions are not allowed")
+
+    if targets[-1] not in TVM_TARGETS:
+        raise TVMCException(f"The last target needs to be a TVM target. Choices: {TVM_TARGETS}")
+
+    tvm_targets = [t for t in targets if t in TVM_TARGETS]
+    tvm_targets_count = len(tvm_targets)
+    if tvm_targets_count > 1:
+        verbose_tvm_targets = ", ".join(tvm_targets)
+        raise TVMCException(
+            f"Only one of the following targets can be used at a time. "
+            "Found {tvm_targets_count}: {verbose_tvm_targets}."
+        )
+
+
+def tokenize_target(target):

Review comment:
       Seems not necessary to be a standalone function.

##########
File path: python/tvm/driver/tvmc/compiler.py
##########
@@ -191,22 +198,21 @@ def compile_model(
 
         if use_autoscheduler:
             with auto_scheduler.ApplyHistoryBest(tuning_records):
-                with tvm.transform.PassContext(
-                    opt_level=3, config={"relay.backend.use_auto_scheduler": True}
-                ):
+                config["relay.backend.use_auto_scheduler"] = True
+                with tvm.transform.PassContext(opt_level=3, config=config):

Review comment:
       Why this change?




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