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/09 20:49:41 UTC

[GitHub] [tvm] mehrdadh opened a new pull request, #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

mehrdadh opened a new pull request, #11250:
URL: https://github.com/apache/tvm/pull/11250

   This PR does the following changes:
   1. Adds a dispatcher where we can force a relay module to build by a specific schedule name
   2. Adds relay level test for various conv2d schedules
   3. Moves aot test utils to python package to be accessible from python package.


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


[GitHub] [tvm] mehrdadh commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r874219886


##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -708,31 +708,52 @@ def compile_models(
 
     compiled_mods = list()
     for model in models:
-        with tvm.transform.PassContext(opt_level=3, config=config):
-            # TODO(Mousius) - Remove once executor/runtime are fully removed from Target
-            if use_runtime_executor:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    target,
-                    executor=executor,
-                    runtime=runtime,
-                    workspace_memory_pools=workspace_memory_pools,
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
-            else:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    tvm.target.Target(target, host=target),
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
+        if schedule_name:

Review Comment:
   I don't think there's a way to do that from AOTExecutorFactoryModule



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


[GitHub] [tvm] mehrdadh commented on pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on PR #11250:
URL: https://github.com/apache/tvm/pull/11250#issuecomment-1121568069

   cc @mkatanbaf 


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


[GitHub] [tvm] ekalda commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
ekalda commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r873863778


##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):
+        super(ApplyFixedConfig, self).__init__()
+        self._schedule_name = schedule_name
+        self._tasks = tasks
+        self.workload = None
+
+    def _query_inside(self, target, workload):
+        """Override query"""
+        self.workload = workload
+
+        # Creat a config from correct task

Review Comment:
   nit:
   ```suggestion
           # Create a config from correct task
   ```



##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -708,31 +708,52 @@ def compile_models(
 
     compiled_mods = list()
     for model in models:
-        with tvm.transform.PassContext(opt_level=3, config=config):
-            # TODO(Mousius) - Remove once executor/runtime are fully removed from Target
-            if use_runtime_executor:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    target,
-                    executor=executor,
-                    runtime=runtime,
-                    workspace_memory_pools=workspace_memory_pools,
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
-            else:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    tvm.target.Target(target, host=target),
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
+        if schedule_name:
+            # Testing with deterministic schedule
+            task_list = autotvm.task.extract_from_program(
+                model.module, target=target, params=model.params
+            )
+            with tvm.autotvm.apply_fixed_config(task_list, schedule_name):
+                with tvm.transform.PassContext(opt_level=3, config=config):
+                    if use_runtime_executor:
+                        executor_factory = tvm.relay.build(
+                            model.module,
+                            target,
+                            executor=executor,
+                            runtime=runtime,
+                            workspace_memory_pools=workspace_memory_pools,
+                            params=model.params,
+                            mod_name=model.name,
+                        )
+                        compiled_mods.append(
+                            AOTCompiledTestModel(model=model, executor_factory=executor_factory)
+                        )

Review Comment:
   Missing else block?



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


[GitHub] [tvm] mehrdadh commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r874139695


##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):
+        super(ApplyFixedConfig, self).__init__()
+        self._schedule_name = schedule_name
+        self._tasks = tasks
+        self.workload = None
+
+    def _query_inside(self, target, workload):
+        """Override query"""
+        self.workload = workload
+
+        # Creat a config from correct task
+        for task in self._tasks:
+            if task.name == workload[0]:
+                config = task.config_space.get(0)
+                break
+
+        if not config:
+            raise RuntimeError(
+                "workload: %s does not exist in %s" % (str(workload), str(self._tasks))
+            )
+        # Add low cost to the target schedule and high cost to others.
+        if workload[0] == self._schedule_name:
+            config.cost = 0.000001

Review Comment:
   done



##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):

Review Comment:
   can you explain why?



##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -38,12 +36,13 @@
 import tvm
 from tvm import relay
 from tvm import te
+from tvm import autotvm

Review Comment:
   agreed, I splitted that file into `python/tvm/testing/aot.py` and `python/tvm/micro/testing/aot_test_utils.py`



##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.

Review Comment:
   added more details.



##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -708,31 +708,52 @@ def compile_models(
 
     compiled_mods = list()
     for model in models:
-        with tvm.transform.PassContext(opt_level=3, config=config):
-            # TODO(Mousius) - Remove once executor/runtime are fully removed from Target
-            if use_runtime_executor:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    target,
-                    executor=executor,
-                    runtime=runtime,
-                    workspace_memory_pools=workspace_memory_pools,
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
-            else:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    tvm.target.Target(target, host=target),
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
+        if schedule_name:
+            # Testing with deterministic schedule
+            task_list = autotvm.task.extract_from_program(
+                model.module, target=target, params=model.params
+            )
+            with tvm.autotvm.apply_fixed_config(task_list, schedule_name):
+                with tvm.transform.PassContext(opt_level=3, config=config):
+                    if use_runtime_executor:
+                        executor_factory = tvm.relay.build(
+                            model.module,
+                            target,
+                            executor=executor,
+                            runtime=runtime,
+                            workspace_memory_pools=workspace_memory_pools,
+                            params=model.params,
+                            mod_name=model.name,
+                        )
+                        compiled_mods.append(
+                            AOTCompiledTestModel(model=model, executor_factory=executor_factory)
+                        )

Review Comment:
   added



##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):
+        super(ApplyFixedConfig, self).__init__()
+        self._schedule_name = schedule_name
+        self._tasks = tasks
+        self.workload = None
+
+    def _query_inside(self, target, workload):
+        """Override query"""
+        self.workload = workload
+
+        # Creat a config from correct task

Review Comment:
   fixed, thanks!



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


[GitHub] [tvm] areusch commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
areusch commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r875348681


##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):

Review Comment:
   i think this would get called for each function being scheduled. so if the Relay program was more complex (e.g. task extraction found multiple tasks), then we'd want to specify a schedule for each.



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


[GitHub] [tvm] areusch commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
areusch commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r877557927


##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -708,31 +708,52 @@ def compile_models(
 
     compiled_mods = list()
     for model in models:
-        with tvm.transform.PassContext(opt_level=3, config=config):
-            # TODO(Mousius) - Remove once executor/runtime are fully removed from Target
-            if use_runtime_executor:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    target,
-                    executor=executor,
-                    runtime=runtime,
-                    workspace_memory_pools=workspace_memory_pools,
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
-            else:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    tvm.target.Target(target, host=target),
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
+        if schedule_name:

Review Comment:
   i was thinking we could do something like search the output function names 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


[GitHub] [tvm] areusch commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
areusch commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r877555837


##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.

Review Comment:
   i think the comment no longer makes sense now that we are accepting an array of schedule names, but can fix that in a follow-on.



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


[GitHub] [tvm] mehrdadh commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r877561937


##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -708,31 +708,52 @@ def compile_models(
 
     compiled_mods = list()
     for model in models:
-        with tvm.transform.PassContext(opt_level=3, config=config):
-            # TODO(Mousius) - Remove once executor/runtime are fully removed from Target
-            if use_runtime_executor:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    target,
-                    executor=executor,
-                    runtime=runtime,
-                    workspace_memory_pools=workspace_memory_pools,
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
-            else:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    tvm.target.Target(target, host=target),
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
+        if schedule_name:

Review Comment:
   We can iterate on this in a follow up PR. thanks!



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


[GitHub] [tvm] mehrdadh commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r877561642


##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.

Review Comment:
   will fix that in a follow up PR.



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


[GitHub] [tvm] mehrdadh merged pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh merged PR #11250:
URL: https://github.com/apache/tvm/pull/11250


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


[GitHub] [tvm] areusch commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
areusch commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r871662035


##########
python/tvm/topi/arm_cpu/conv2d_int8.py:
##########
@@ -126,7 +126,7 @@ def is_int8_hw_support(data_dtype, kernel_dtype):
     # 3) Check target
     is_target_support = is_neon_available() or is_dotprod_available()
 
-    return is_dtype_support and is_llvm_support
+    return is_dtype_support and is_llvm_support and is_target_support

Review Comment:
   is this one a carryover from #11193 ?



##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):

Review Comment:
   for schedule_name, should we be taking a mapping here, e.g. a dict mapping compute function name to schedule or something?



##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -38,12 +36,13 @@
 import tvm
 from tvm import relay
 from tvm import te
+from tvm import autotvm

Review Comment:
   i'm wondering if we shold place this file in python/tvm/testing instead? AOT is not specific to micro. we could also split the Corstone stuff into micro.



##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.

Review Comment:
   could you explain why this is different from ApplyConfig here?



##########
python/tvm/autotvm/task/dispatcher.py:
##########
@@ -178,6 +178,50 @@ def update(self, target, workload, cfg):
         self._config = cfg
 
 
+class ApplyFixedConfig(DispatchContext):
+    """Apply a config of a deterministic schedule.
+
+    Parameters
+    ----------
+    tasks : list[tvm.autotvm.task.task.Task]
+        List of autoTVM tasks.
+    schedule_name : str
+        Name of schedule to use.
+    """
+
+    def __init__(self, tasks, schedule_name: str):
+        super(ApplyFixedConfig, self).__init__()
+        self._schedule_name = schedule_name
+        self._tasks = tasks
+        self.workload = None
+
+    def _query_inside(self, target, workload):
+        """Override query"""
+        self.workload = workload
+
+        # Creat a config from correct task
+        for task in self._tasks:
+            if task.name == workload[0]:
+                config = task.config_space.get(0)
+                break
+
+        if not config:
+            raise RuntimeError(
+                "workload: %s does not exist in %s" % (str(workload), str(self._tasks))
+            )
+        # Add low cost to the target schedule and high cost to others.
+        if workload[0] == self._schedule_name:
+            config.cost = 0.000001

Review Comment:
   better is 1e-6



##########
tests/python/relay/strategy/arm_cpu/test_conv2d_nhwc.py:
##########
@@ -0,0 +1,163 @@
+# 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.
+import sys
+import numpy as np
+import pytest
+import tvm
+import tvm.testing
+from tvm import relay
+from tvm.micro.testing.aot_test_utils import (
+    AOTTestModel,
+    AOT_CORSTONE300_RUNNER,
+    generate_ref_data,
+    compile_and_run,
+)
+
+
+class BasicConv2dTests:
+    @tvm.testing.requires_corstone300
+    def test_conv2d(
+        self,
+        data_shape,
+        kernel_size,
+        kernel_layout,
+        num_filter,
+        strides,
+        padding,
+        dilation,
+        dtype,
+        schedule_name,
+    ):
+        """Test a subgraph with a single conv2d operator."""
+        ishape = data_shape
+        wshape = (*kernel_size, data_shape[-1], num_filter)
+
+        weight_data = np.random.randint(low=-10, high=10, size=wshape, dtype=dtype)
+
+        input0 = relay.var("input", relay.TensorType(ishape, dtype))
+        weight0 = relay.const(weight_data)
+        out0 = relay.op.nn.conv2d(
+            input0,
+            weight0,
+            kernel_size=kernel_size,
+            strides=strides,
+            padding=padding,
+            dilation=(dilation, dilation),
+            data_layout="NHWC",
+            kernel_layout="HWIO",
+            out_dtype="int32",
+            out_layout="NHWC",
+        )
+        ref_mod = tvm.IRModule.from_expr(relay.Function([input0], out0))
+
+        input1 = relay.var("input", relay.TensorType(ishape, dtype))
+
+        if kernel_layout == "HWOI":
+            weight1 = relay.const(np.moveaxis(weight_data, 2, -1))
+        elif kernel_layout == "HWIO":
+            weight1 = relay.const(weight_data)
+
+        out1 = relay.op.nn.conv2d(
+            input1,
+            weight1,
+            kernel_size=kernel_size,
+            strides=strides,
+            padding=padding,
+            dilation=(dilation, dilation),
+            data_layout="NHWC",
+            kernel_layout=kernel_layout,
+            out_dtype="int32",
+            out_layout="NHWC",
+        )
+        mod = tvm.IRModule.from_expr(relay.Function([input1], out1))
+
+        inputs = {"input": np.random.randint(low=-128, high=127, size=ishape, dtype=dtype)}
+        output_list = generate_ref_data(ref_mod, inputs)
+
+        compile_and_run(
+            AOTTestModel(module=mod, inputs=inputs, outputs=output_list),
+            runner=AOT_CORSTONE300_RUNNER,
+            interface_api="c",
+            use_unpacked_api=True,
+            target_opts={
+                "-keys": "arm_cpu",
+                "-mcpu": "cortex-m7",
+            },
+            schedule_name=schedule_name,
+        )
+
+
+class TestConv2d_DSP_HWOI(BasicConv2dTests):
+    """This test is for conv2d_nhwc_dsp.arm_cpu schedule."""
+
+    data_shape, kernel_size, num_filter, strides, padding, dilation = tvm.testing.parameters(
+        # TODO(mehrdadh): Fails due to https://github.com/apache/tvm/issues/11216
+        # ((1, 32, 32, 1), (3, 3), 12, 1, 0, 1),
+        # ((1, 32, 10, 3), (3, 3), 16, 1, 0, 1),
+        # ((1, 49, 10, 1), (10, 4), 64, (2, 1), (4, 1, 5, 1), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 2),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (1, 1, 2, 2), 2),
+        # from Keyword Spotting model from MLPerfTiny models
+        # TODO(mehrdad): Fails due to https://github.com/apache/tvm/issues/11216
+        # ((1, 49, 10, 1), (10, 4), 64, (2, 2), (4, 1, 5, 1), 1),
+        # from Visual Wake Word model from MLPerfTiny models
+        # TODO(mehrdadh): fails due to https://github.com/apache/tvm/issues/11216
+        # ((1, 96, 96, 3), (3, 3), 8, (2, 2), (0, 0, 1, 1), 1),
+        # from Image Classification model from MLPerfTiny models
+        ((1, 16, 16, 32), (1, 1), 64, (2, 2), 0, 1),
+        ((4, 16, 16, 8), (5, 5), 8, 2, (0, 4, 4, 0), 1),
+        ((4, 16, 16, 8), (5, 5), 16, 2, (0, 4, 4, 0), 1),
+        ((4, 16, 16, 8), (5, 5), 8, 2, 0, 1),
+        ((4, 16, 16, 8), (5, 5), 16, 2, 0, 1),
+        ((1, 16, 16, 8), (3, 3), 16, 2, (0, 0, 1, 1), 1),
+        ((1, 16, 16, 8), (3, 3), 16, 2, (1, 1, 2, 2), 1),
+        ((1, 16, 16, 8), (5, 5), 16, 2, (3, 3, 2, 2), 1),
+        ((1, 16, 16, 8), (3, 3), 16, 2, (0, 1, 2, 3), 1),
+    )
+    dtype = tvm.testing.parameter("int8", "int16")
+    kernel_layout = tvm.testing.parameter("HWOI")
+    schedule_name = tvm.testing.parameter("conv2d_nhwc_dsp.arm_cpu")
+
+
+class TestConv2d_HWIO(BasicConv2dTests):
+    """This test is for conv2d_nhwc_spatial_pack.arm_cpu schedule."""
+
+    data_shape, kernel_size, num_filter, strides, padding, dilation = tvm.testing.parameters(
+        ((1, 32, 32, 1), (3, 3), 12, 1, 0, 1),
+        ((1, 32, 10, 3), (3, 3), 16, 1, 0, 1),
+        ((1, 49, 10, 1), (10, 4), 64, (2, 1), (4, 1, 5, 1), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 2),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (1, 1, 2, 2), 2),
+    )
+    dtype = tvm.testing.parameter("int8", "int16")
+    kernel_layout = tvm.testing.parameter("HWIO")
+    schedule_name = tvm.testing.parameter("conv2d_nhwc_spatial_pack.arm_cpu")
+
+
+# TODO(mehrdadh): Add test for conv2d_NHWC_quantized_native.arm_cpu

Review Comment:
   prefer to reference a GH issue rather than a person if possible



##########
python/tvm/micro/testing/aot_test_utils.py:
##########
@@ -708,31 +708,52 @@ def compile_models(
 
     compiled_mods = list()
     for model in models:
-        with tvm.transform.PassContext(opt_level=3, config=config):
-            # TODO(Mousius) - Remove once executor/runtime are fully removed from Target
-            if use_runtime_executor:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    target,
-                    executor=executor,
-                    runtime=runtime,
-                    workspace_memory_pools=workspace_memory_pools,
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
-            else:
-                executor_factory = tvm.relay.build(
-                    model.module,
-                    tvm.target.Target(target, host=target),
-                    params=model.params,
-                    mod_name=model.name,
-                )
-                compiled_mods.append(
-                    AOTCompiledTestModel(model=model, executor_factory=executor_factory)
-                )
+        if schedule_name:

Review Comment:
   could we add some kind of assert here to verify that tvm.relay.build actually chose this schedule?



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


[GitHub] [tvm] mehrdadh commented on pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on PR #11250:
URL: https://github.com/apache/tvm/pull/11250#issuecomment-1124382373

   cc @areusch 


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


[GitHub] [tvm] mehrdadh commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r874187865


##########
tests/python/relay/strategy/arm_cpu/test_conv2d_nhwc.py:
##########
@@ -0,0 +1,163 @@
+# 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.
+import sys
+import numpy as np
+import pytest
+import tvm
+import tvm.testing
+from tvm import relay
+from tvm.micro.testing.aot_test_utils import (
+    AOTTestModel,
+    AOT_CORSTONE300_RUNNER,
+    generate_ref_data,
+    compile_and_run,
+)
+
+
+class BasicConv2dTests:
+    @tvm.testing.requires_corstone300
+    def test_conv2d(
+        self,
+        data_shape,
+        kernel_size,
+        kernel_layout,
+        num_filter,
+        strides,
+        padding,
+        dilation,
+        dtype,
+        schedule_name,
+    ):
+        """Test a subgraph with a single conv2d operator."""
+        ishape = data_shape
+        wshape = (*kernel_size, data_shape[-1], num_filter)
+
+        weight_data = np.random.randint(low=-10, high=10, size=wshape, dtype=dtype)
+
+        input0 = relay.var("input", relay.TensorType(ishape, dtype))
+        weight0 = relay.const(weight_data)
+        out0 = relay.op.nn.conv2d(
+            input0,
+            weight0,
+            kernel_size=kernel_size,
+            strides=strides,
+            padding=padding,
+            dilation=(dilation, dilation),
+            data_layout="NHWC",
+            kernel_layout="HWIO",
+            out_dtype="int32",
+            out_layout="NHWC",
+        )
+        ref_mod = tvm.IRModule.from_expr(relay.Function([input0], out0))
+
+        input1 = relay.var("input", relay.TensorType(ishape, dtype))
+
+        if kernel_layout == "HWOI":
+            weight1 = relay.const(np.moveaxis(weight_data, 2, -1))
+        elif kernel_layout == "HWIO":
+            weight1 = relay.const(weight_data)
+
+        out1 = relay.op.nn.conv2d(
+            input1,
+            weight1,
+            kernel_size=kernel_size,
+            strides=strides,
+            padding=padding,
+            dilation=(dilation, dilation),
+            data_layout="NHWC",
+            kernel_layout=kernel_layout,
+            out_dtype="int32",
+            out_layout="NHWC",
+        )
+        mod = tvm.IRModule.from_expr(relay.Function([input1], out1))
+
+        inputs = {"input": np.random.randint(low=-128, high=127, size=ishape, dtype=dtype)}
+        output_list = generate_ref_data(ref_mod, inputs)
+
+        compile_and_run(
+            AOTTestModel(module=mod, inputs=inputs, outputs=output_list),
+            runner=AOT_CORSTONE300_RUNNER,
+            interface_api="c",
+            use_unpacked_api=True,
+            target_opts={
+                "-keys": "arm_cpu",
+                "-mcpu": "cortex-m7",
+            },
+            schedule_name=schedule_name,
+        )
+
+
+class TestConv2d_DSP_HWOI(BasicConv2dTests):
+    """This test is for conv2d_nhwc_dsp.arm_cpu schedule."""
+
+    data_shape, kernel_size, num_filter, strides, padding, dilation = tvm.testing.parameters(
+        # TODO(mehrdadh): Fails due to https://github.com/apache/tvm/issues/11216
+        # ((1, 32, 32, 1), (3, 3), 12, 1, 0, 1),
+        # ((1, 32, 10, 3), (3, 3), 16, 1, 0, 1),
+        # ((1, 49, 10, 1), (10, 4), 64, (2, 1), (4, 1, 5, 1), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 2),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (1, 1, 2, 2), 2),
+        # from Keyword Spotting model from MLPerfTiny models
+        # TODO(mehrdad): Fails due to https://github.com/apache/tvm/issues/11216
+        # ((1, 49, 10, 1), (10, 4), 64, (2, 2), (4, 1, 5, 1), 1),
+        # from Visual Wake Word model from MLPerfTiny models
+        # TODO(mehrdadh): fails due to https://github.com/apache/tvm/issues/11216
+        # ((1, 96, 96, 3), (3, 3), 8, (2, 2), (0, 0, 1, 1), 1),
+        # from Image Classification model from MLPerfTiny models
+        ((1, 16, 16, 32), (1, 1), 64, (2, 2), 0, 1),
+        ((4, 16, 16, 8), (5, 5), 8, 2, (0, 4, 4, 0), 1),
+        ((4, 16, 16, 8), (5, 5), 16, 2, (0, 4, 4, 0), 1),
+        ((4, 16, 16, 8), (5, 5), 8, 2, 0, 1),
+        ((4, 16, 16, 8), (5, 5), 16, 2, 0, 1),
+        ((1, 16, 16, 8), (3, 3), 16, 2, (0, 0, 1, 1), 1),
+        ((1, 16, 16, 8), (3, 3), 16, 2, (1, 1, 2, 2), 1),
+        ((1, 16, 16, 8), (5, 5), 16, 2, (3, 3, 2, 2), 1),
+        ((1, 16, 16, 8), (3, 3), 16, 2, (0, 1, 2, 3), 1),
+    )
+    dtype = tvm.testing.parameter("int8", "int16")
+    kernel_layout = tvm.testing.parameter("HWOI")
+    schedule_name = tvm.testing.parameter("conv2d_nhwc_dsp.arm_cpu")
+
+
+class TestConv2d_HWIO(BasicConv2dTests):
+    """This test is for conv2d_nhwc_spatial_pack.arm_cpu schedule."""
+
+    data_shape, kernel_size, num_filter, strides, padding, dilation = tvm.testing.parameters(
+        ((1, 32, 32, 1), (3, 3), 12, 1, 0, 1),
+        ((1, 32, 10, 3), (3, 3), 16, 1, 0, 1),
+        ((1, 49, 10, 1), (10, 4), 64, (2, 1), (4, 1, 5, 1), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, 0, 1),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (0, 2, 2, 0), 2),
+        ((1, 32, 32, 16), (3, 3), 16, 1, (1, 1, 2, 2), 2),
+    )
+    dtype = tvm.testing.parameter("int8", "int16")
+    kernel_layout = tvm.testing.parameter("HWIO")
+    schedule_name = tvm.testing.parameter("conv2d_nhwc_spatial_pack.arm_cpu")
+
+
+# TODO(mehrdadh): Add test for conv2d_NHWC_quantized_native.arm_cpu

Review Comment:
   removed them.



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


[GitHub] [tvm] mehrdadh commented on a diff in pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on code in PR #11250:
URL: https://github.com/apache/tvm/pull/11250#discussion_r874184403


##########
python/tvm/topi/arm_cpu/conv2d_int8.py:
##########
@@ -126,7 +126,7 @@ def is_int8_hw_support(data_dtype, kernel_dtype):
     # 3) Check target
     is_target_support = is_neon_available() or is_dotprod_available()
 
-    return is_dtype_support and is_llvm_support
+    return is_dtype_support and is_llvm_support and is_target_support

Review Comment:
   it was, fixed it.



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


[GitHub] [tvm] mehrdadh commented on pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
mehrdadh commented on PR #11250:
URL: https://github.com/apache/tvm/pull/11250#issuecomment-1128214360

   @areusch PTAL, thanks


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


[GitHub] [tvm] areusch commented on pull request #11250: [microTVM][ARM] Add Relay tests for conv2d registered schedules

Posted by GitBox <gi...@apache.org>.
areusch commented on PR #11250:
URL: https://github.com/apache/tvm/pull/11250#issuecomment-1129415023

   @mehrdadh just replied on the one thread, lmk if that makes sense


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