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/16 01:20:21 UTC

[GitHub] [incubator-tvm] merrymercy opened a new pull request #6488: [AutoScheduler] Tutorial on how to auto-schedule a subgraph for CPU

merrymercy opened a new pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488


   Add a tutorial on how to auto-schedule a subgraph for CPU.


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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489127250



##########
File path: python/tvm/auto_scheduler/auto_schedule.py
##########
@@ -156,16 +159,41 @@ def __init__(
         )
 
 
+def create_task(func, args, target, target_host=None, hardware_params=None):

Review comment:
       Python does not support constructor overloading, so I have to create a new function for this.
   
   I think both `auto_scheduler.create_task` and `auto_scheduler.task.create` are okay.




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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489884416



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,173 @@
+# 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.
+"""
+Auto-scheduling matrix multiplication for CPU
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the existing :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   and do more analyses later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it found.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)

Review comment:
       I will merge this and test the docstring link on the tvm website.
   If it is not working, I can add more explanations in the next PR, which is a similar tutorial for GPU.




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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489133644



##########
File path: python/tvm/auto_scheduler/auto_schedule.py
##########
@@ -156,16 +159,41 @@ def __init__(
         )
 
 
+def create_task(func, args, target, target_host=None, hardware_params=None):
+    """Create a search task
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The function that returns the compute declaration Tensors.
+        Can be the a function or the function name.
+    args : Args

Review comment:
       ```suggestion
       args : Union[Tuple[Any, ...], List[Any]]
   ```

##########
File path: python/tvm/auto_scheduler/auto_schedule.py
##########
@@ -156,16 +159,41 @@ def __init__(
         )
 
 
+def create_task(func, args, target, target_host=None, hardware_params=None):
+    """Create a search task
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The function that returns the compute declaration Tensors.
+        Can be the a function or the function name.
+    args : Args
+        The args of the function.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this search task.
+    hardware_params : Optional[HardwareParams]
+        Hardware parameters used in this search task.
+
+    Returns
+    -------
+        task : the created task

Review comment:
       ```suggestion
           SearchTask: the created task
   ```

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   or do more analysis later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it founds.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)
+
+######################################################################
+# We can lower schedule to see the IR after auto-scheduling.
+# The auto-scheduler correctly performs optimizations including multi-level tiling,
+# parallelization, vectorization, unrolling and fusion.
+
+print(tvm.lower(sch, args, simple_mode=True))
+
+######################################################################
+# Check correctness
+# ^^^^^^^^^^^^^^^^^
+# We build the binary and check its correctness
+
+func = tvm.build(sch, args)
+a_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+b_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+c_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+d_np = a_np.dot(b_np) + c_np
+
+d_tvm = tvm.nd.empty(d_np.shape)
+func(tvm.nd.array(a_np), tvm.nd.array(b_np), tvm.nd.array(c_np), d_tvm)
+
+tvm.testing.assert_allclose(d_np, d_tvm.asnumpy(), rtol=1e-3)
+
+######################################################################
+# Using the record file
+# ^^^^^^^^^^^^^^^^^^^^^
+# During the search, all measuremnt records are dumpped into the record
+# file "matmul.json". The measurement records can be used to resume the
+# search, re-apply search results and other analysis.

Review comment:
       It would be better to also show how to resume the search with existing logs.




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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489884416



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,173 @@
+# 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.
+"""
+Auto-scheduling matrix multiplication for CPU
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the existing :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   and do more analyses later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it found.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)

Review comment:
       I merged this first and will see whether the docstring link on the tvm website works.
   If it is not working, I can add more explanations in the next PR, which is a similar tutorial for GPU.




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



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #6488: [TUTORIAL] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
jcf94 commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489112695



##########
File path: python/tvm/auto_scheduler/auto_schedule.py
##########
@@ -156,16 +159,41 @@ def __init__(
         )
 
 
+def create_task(func, args, target, target_host=None, hardware_params=None):

Review comment:
       I'm thinking about what's the different of calling `SearchTask()` directly, seems they just have different input parameters.
   
   I notice that AutoTVM used an api of `autotvm.task.create()`, should we keep them the same?




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



[GitHub] [incubator-tvm] merrymercy merged pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy merged pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488


   


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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy edited a comment on pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#issuecomment-693519885


   @mbaret Good catch! It is actually a small bug introduced during the upstream. Now the bug is fixed and you can give it another try.
   One of the most important advantages of Ansor is just to try different fusion strategies for an arbitrary subgraph. So it should try both inline and no inline.


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



[GitHub] [incubator-tvm] mbaret commented on pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
mbaret commented on pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#issuecomment-693550734


   Ah nice, it's working as I expected now.


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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy edited a comment on pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#issuecomment-693519885


   @mbaret Good catch! It is actually a small bug introduced during the upstream. Now the bug is fixed and you can give it another try.
   Ansor is designed to work with arbitrary subgraphs and it should fuse (inline) them correctly.
   


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



[GitHub] [incubator-tvm] tmoreau89 commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
tmoreau89 commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489127689



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"

Review comment:
       remove the/a

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   or do more analysis later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it founds.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)
+
+######################################################################
+# We can lower schedule to see the IR after auto-scheduling.

Review comment:
       lower the schedule

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   or do more analysis later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it founds.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)
+
+######################################################################
+# We can lower schedule to see the IR after auto-scheduling.
+# The auto-scheduler correctly performs optimizations including multi-level tiling,
+# parallelization, vectorization, unrolling and fusion.
+
+print(tvm.lower(sch, args, simple_mode=True))
+
+######################################################################
+# Check correctness
+# ^^^^^^^^^^^^^^^^^
+# We build the binary and check its correctness
+
+func = tvm.build(sch, args)
+a_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+b_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+c_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+d_np = a_np.dot(b_np) + c_np
+
+d_tvm = tvm.nd.empty(d_np.shape)
+func(tvm.nd.array(a_np), tvm.nd.array(b_np), tvm.nd.array(c_np), d_tvm)
+
+tvm.testing.assert_allclose(d_np, d_tvm.asnumpy(), rtol=1e-3)
+
+######################################################################
+# Using the record file
+# ^^^^^^^^^^^^^^^^^^^^^
+# During the search, all measuremnt records are dumpped into the record
+# file "matmul.json". The measurement records can be used to resume the
+# search, re-apply search results and other analysis.
+#
+# Here we show an example where we load the best schedule from a file,
+# print the equivalent python schedule API, and build the binary again.
+
+# Load the measuremnt record for the best schedule
+inp, res = auto_scheduler.load_best("matmul.json", task.workload_key)
+
+# Print equivalent python schedule API. This can be used for debugging and
+# learning the behavior of auto-scheduler.

Review comment:
       "of the autoscheduler"

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   or do more analysis later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it founds.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)
+
+######################################################################
+# We can lower schedule to see the IR after auto-scheduling.
+# The auto-scheduler correctly performs optimizations including multi-level tiling,
+# parallelization, vectorization, unrolling and fusion.
+
+print(tvm.lower(sch, args, simple_mode=True))
+
+######################################################################
+# Check correctness
+# ^^^^^^^^^^^^^^^^^
+# We build the binary and check its correctness
+
+func = tvm.build(sch, args)
+a_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+b_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+c_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+d_np = a_np.dot(b_np) + c_np
+
+d_tvm = tvm.nd.empty(d_np.shape)
+func(tvm.nd.array(a_np), tvm.nd.array(b_np), tvm.nd.array(c_np), d_tvm)
+
+tvm.testing.assert_allclose(d_np, d_tvm.asnumpy(), rtol=1e-3)
+
+######################################################################
+# Using the record file
+# ^^^^^^^^^^^^^^^^^^^^^
+# During the search, all measuremnt records are dumpped into the record
+# file "matmul.json". The measurement records can be used to resume the
+# search, re-apply search results and other analysis.

Review comment:
       and perform other analyses. 

##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   or do more analysis later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it founds.

Review comment:
       it found




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



[GitHub] [incubator-tvm] binarybana commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
binarybana commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489763585



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,173 @@
+# 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.
+"""
+Auto-scheduling matrix multiplication for CPU
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the existing :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   and do more analyses later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it found.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)

Review comment:
       It'd be nice to briefly describe what the return values of `sch` and `args` _are_ here. I'm assuming that `sch` is a schedule, but is it encoded in TIR, or some other data structure? Same with `args`. 




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



[GitHub] [incubator-tvm] masahi commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489190200



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 

Review comment:
       existing




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



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
jcf94 commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489851333



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,173 @@
+# 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.
+"""
+Auto-scheduling matrix multiplication for CPU
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the existing :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   and do more analyses later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it found.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)

Review comment:
       If you check the doc string of the `auto_scheduler.auto_schedule()`, you'll find the explaination of the return values: A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.




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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489602513



##########
File path: tutorials/auto_scheduler/tune_matmul_x86.py
##########
@@ -0,0 +1,136 @@
+# 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.
+"""
+Using the template-free auto-scheduler on CPU 
+=============================================
+**Author**: `Lianmin Zheng <https://github.com/merrymercy>`_, \
+            `Chengfan Jia <https://github.com/jcf94/>`_
+
+Different from the exiting :ref:`autotvm <tutorials-autotvm-sec>` which relies on 
+manual templates to define the search space, the auto-scheduler does not require any templates.
+The auto-scheduler is template-free, so users only need to write the computation declaration without
+any schedule commands or templates.
+The auto-scheduler can automatically generate a large
+search space and find a good schedule in the space.
+
+We use matrix multiplication as an example in this tutorial.
+"""
+
+import numpy as np
+import tvm
+from tvm import te, testing, auto_scheduler
+
+######################################################################
+# Define the computation
+# ^^^^^^^^^^^^^^^^^^^^^^
+# To begin with, we define the computation of a matmul with bias add.
+# The function should return the list of input/output tensors.
+# From these tensors, the auto-scheduler can get the whole computational graph.
+
+
+@auto_scheduler.register_workload
+def matmul_add(N, L, M, dtype):
+    A = te.placeholder((N, L), name="A", dtype=dtype)
+    B = te.placeholder((L, M), name="B", dtype=dtype)
+    C = te.placeholder((N, M), name="C", dtype=dtype)
+
+    k = te.reduce_axis((0, L), name="k")
+    matmul = te.compute((N, M), lambda i, j: te.sum(A[i, k] * B[k, j], axis=k), name="matmul")
+    out = te.compute((N, M), lambda i, j: matmul[i, j] + C[i, j], name="out")
+
+    return [A, B, C, out]
+
+
+######################################################################
+# Create the search task
+# ^^^^^^^^^^^^^^^^^^^^^^
+# We then create the a search task with N=L=M=128 and dtype="float32"
+
+target = tvm.target.Target("llvm")
+task = auto_scheduler.create_task(matmul_add, (128, 128, 128, "float32"), target)
+
+# Inspect the computational graph
+print(task.compute_dag)
+
+######################################################################
+# Next, we set parameters for the auto-scheduler.
+#
+# * `num_measure_trials` is the number of measurement trials we can use during the search.
+#   We only make 10 trials in this tutorial for a fast demonstration. In practice, 1000 is a
+#   good value for the search to converge. You can do more trials according to your time budget.
+# * In addition, we use `RecordToFile` to dump measurement records into a file `matmul.json`.
+#   The measurement records can be used to query the history best, resume the search,
+#   or do more analysis later.
+# * see :any:`auto_schedule.TuningOptions`: for more parameters
+
+tune_option = auto_scheduler.TuningOptions(
+    num_measure_trials=10, measure_callbacks=[auto_scheduler.RecordToFile("matmul.json")]
+)
+
+######################################################################
+# Run the search
+# ^^^^^^^^^^^^^^
+# Now we get all inputs ready. Pretty simple, isn't it?
+# We can kick off the search and let the auto-scheduler do its magic.
+# After some measurement trials, it will return the best schedule it founds.
+
+sch, args = auto_scheduler.auto_schedule(task, tuning_options=tune_option)
+
+######################################################################
+# We can lower schedule to see the IR after auto-scheduling.
+# The auto-scheduler correctly performs optimizations including multi-level tiling,
+# parallelization, vectorization, unrolling and fusion.
+
+print(tvm.lower(sch, args, simple_mode=True))
+
+######################################################################
+# Check correctness
+# ^^^^^^^^^^^^^^^^^
+# We build the binary and check its correctness
+
+func = tvm.build(sch, args)
+a_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+b_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+c_np = np.random.uniform(size=(128, 128)).astype(np.float32)
+d_np = a_np.dot(b_np) + c_np
+
+d_tvm = tvm.nd.empty(d_np.shape)
+func(tvm.nd.array(a_np), tvm.nd.array(b_np), tvm.nd.array(c_np), d_tvm)
+
+tvm.testing.assert_allclose(d_np, d_tvm.asnumpy(), rtol=1e-3)
+
+######################################################################
+# Using the record file
+# ^^^^^^^^^^^^^^^^^^^^^
+# During the search, all measuremnt records are dumpped into the record
+# file "matmul.json". The measurement records can be used to resume the
+# search, re-apply search results and other analysis.

Review comment:
       The example is added




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



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
jcf94 commented on a change in pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#discussion_r489112695



##########
File path: python/tvm/auto_scheduler/auto_schedule.py
##########
@@ -156,16 +159,41 @@ def __init__(
         )
 
 
+def create_task(func, args, target, target_host=None, hardware_params=None):

Review comment:
       I'm thinking about what's the different of calling `SearchTask()` directly, seems they just have different input parameters.
   
   I notice that AutoTVM used an api of `autotvm.task.create()`, should we keep them the same?
   
   Anyway, it's fine since we may continue to refine our APIs later.




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



[GitHub] [incubator-tvm] merrymercy commented on pull request #6488: [TUTORIAL][ANSOR] Using the template-free auto-scheduler on CPU

Posted by GitBox <gi...@apache.org>.
merrymercy commented on pull request #6488:
URL: https://github.com/apache/incubator-tvm/pull/6488#issuecomment-693519885


   @mbaret Good catch! It is actually a small bug introduced during the upstream.
   Ansor is designed to work with arbitrary subgraph and it should fuse (inline) them correctly.
   


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