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/06/30 06:11:02 UTC

[GitHub] [incubator-tvm] jcf94 opened a new pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Hi all,
   The last PR of Ansor #5883 is not clear enough for reviewers to fully understand our design. After some discussion, we changed our upstream plan to propose a minimal version of Ansor which contains a small but complete framework, so others can get a better understanding of the whole structure of Ansor.
   
   ---
   
   
   In [[RFC] Ansor: An Auto-scheduler for TVM (AutoTVM v2.0)](https://discuss.tvm.ai/t/rfc-ansor-an-auto-scheduler-for-tvm-autotvm-v2-0/7005), we've introduced the **Ansor** auto-scheduler. And we reached an agreement that should replace AutoTVM with Ansor in the end.
   For most existing templates, current Ansor can directly replace them with better performance and less tuning time.
   For other special templates (low-precision, sparse), the plan is to introduce search space customization and gradually rewrite them with Ansor's new API.
   
   This PR introduces a self-contained minimum version of Ansor with most of the bones.
   
   This PR includes the interface of core data structures and an empty search policy that does nothing. More advanced search policy and cost models will be in the next few PRs.
   
   # Infrastructure: A Sketch IR for Schedule Searching
   Different from AutoTVM, whose tuning spaces are composed of predefined parameters, Ansor constructs a search space by manipulating the loop structures of the given compute DAG.
   To enable flexible manipulation of the loop structures, we implemented a lightweight loop structure IR (Intermediate Representation) based on the original TVM IR but specifically for schedule search. The IR is composed of the **state** and **action**, which are defined as follows:
   
   - **State**: A state of schedule search is the loop structure defined by the schedule (i.e., the TVM IR created by `tvm.lower`). See **LoopState** in the Key Data Structure for details.
   
   - **Action**: An action is composed of one or more schedule primitives to manipulate (e.g., split, reorder, fuse) a state. See **TransformStep** in the Key Data Structure for details.
   
   We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
   1. We want fast incremental change to the loop structures;
   2. We want serializable transform history for replay, backtracking, and mutation;
   3. We may create some macro schedule primitives that represent the combination of several TVM schedule primitives
   
   After the search is done, we will lower this to TVM IR with TVM schedule primitives.
   
   # Key Data Structures
   To help build an overview of Ansor, this shows the class relations of some important Ansor data structures:
   <img width="1222" alt="截屏2020-06-30 上午11 17 25" src="https://user-images.githubusercontent.com/12119698/86079551-55da9980-bac3-11ea-9f5f-d51e3dce5dfd.png">
   
   - **ComputeDAG**: Compute declaration graph and its related analysis tools.
   Related source files: `src/ansor/compute_dag.*`, `python/tvm/ansor/compute_dag.py`
   Ansor takes a compute declaration, which could be a single operator or a subgraph, described by `tvm.compute` as an input and converts it to a ComputeDAG.
   ComputeDAG implementation includes a set of analyses such as the total float operations, consumer/producer relations of each operation stage, whether a stage should be tiled/compute inlined, and so on (*some of the analysis will be included in the follow-up PRs*). These analyses can help the search policy to do specific decisions during schedule search process.
   
   - **LoopState**: This defines the "state" for the search problem.
   Related source files: `src/ansor/loop_state.*`, `python/tvm/ansor/loop_state.py`.
   Each LoopState corresponds to a specific schedule for the target ComputeDAG.
   A LoopState consists of: 1. the current loop structure; 2. the transform history to reach this loop structure.
   The loop structure keeps a **preview** of how the schedule will finally look like after lowering (number of iterators, the extent of each iterator, the `compute_at` locations, etc), which can help the search policy to make decisions during the search.
   The transform history is a sequence of `TransformStep` which will finally be mapped to schedule primitives.
   
   - **TransformStep**: This defines the "action" for the search problem, i.e., the schedule primitives for our sketch IR.
   Related files: `src/ansor/transform_step.*`, `python/tvm/ansor/loop_state.py`
   Each step has its corresponding `tvm.te` schedule primitives. We record all TransformSteps for every state as its transform history. After finishing the schedule search, these transform steps will be lowered with their corresponding TVM's schedule primitives.
   *Note*: This PR only contains a small subset of the TransformSteps. The complete set of transform steps will be in the next PRs.
   
   > ComputeDAG is also playing a role of connecting Ansor state system to TVM schedule system. That is, ComputeDAG is able to replay TransformSteps to the final TVM schedule (e.g., `ComputeDAG(state, actions)`).
   
   - **SearchTask**: Meta information and hardware parameters for a specific schedule search task.
   Related source files: `src/ansor/search_task.*`
   This structure includes the target ComputeDAG, device information as well as some hardware parameters obtained from the system or user inputs.
   
   - **SearchPolicy**: The search policy is defined for Ansor to auto-generate a high-performance schedule for different computations.
   Related source files: `src/ansor/search_policy/*`
   A SearchPolicy takes a `SearchTask`, system information and some tuning options as inputs, performs the schedule search, and returns a state with the best performance. The resulting state can be used to apply to TVM schedule later.
   
     Note that in Ansor paper ([https://arxiv.org/abs/2006.06762](https://arxiv.org/abs/2006.06762)), we proposed a sketch generation policy which achieves pretty good results with various workloads on different devices. On the other hand, in this minimum system PR, we only provide an `EmptyPolicy` to illustrate the search policy interface.
   
   # Ansor Minimum System
   
   This is a brief diagram for the Ansor system:
   <img width="1481" alt="截屏2020-06-30 上午10 44 59" src="https://user-images.githubusercontent.com/12119698/86077606-d3e87180-babe-11ea-8c3d-fe52c918db18.png">
   
   1. Define the target computation with TVM `te` API and create a ComputeDAG structure.
   2. Specify the target device, hardware parameters, tuning options and pack those with `ComputeDAG` to create a `SearchTask` structure.
   3. `SearchPolicy` takes the `SearchTask` as input and performs the schedule search. During the search process, `SearchPolicy` will generate multiple candidate states, each of which corresponds to a specific TVM schedule.
   4. Get the best state and use ComputeDAG API to transform it to the final TVM schedule.
   
   In the Ansor system, we use the sketch generation policy (will be brought in later PRs) described in the paper as the default search policy, which should be enough to cover most use cases. Meanwhile, we will have an RFC for a custom rule mechanism that enables user-defined template search to serve the same functionality as the current AutoTVM template. Specifically, we will provide Python APIs for the new IR that is intended to be used by users for sketch customization. They look very similar to existing schedule primitives, as shown in `python/tvm/ansor/loop_state.py`.
   
   Our goal is to make sure Ansor can cover all AutoTVM functionalities while achieving the same or better performance so that the community can gradually switch to Ansor from AutoTVM.
   
   # More Details on LoopState
   
   In this section, we illustrate how a loop state looks like and how does it connect to the current TVM build system.
   
   Take a simple State that includes Split, Fuse and Reorder steps for example:
   
   ```python
   A, B, C = matmul_ansor_test(512, 512, 512)
   dag = ansor.ComputeDAG([A, B, C])
   state = dag.get_init_state()
   i, j, k = state[C].iters
   io, ii = state.split(C, i, [16])
   jo, ji = state.split(C, j, [8])
   state.reorder(C, [io, jo, k, ji, ii])
   fused_it = state.fuse(C, [io, jo])
   ```
   
   First, let's print out a state. It shows the loop structure of the corresponding TVM schedule, the "preview":
   ```python
   >>> print(state)
   
   Placeholder: A, B
   for i.0@j.0@ (0,2048)
     for k (0,512)
       for j.1 (0,8)
         for i.1 (0,16)
           C = ...
   ```
   
   It stores all history transform steps required to reach the current state. We can print the history transform steps as TVM's python schedule API. This can be used for debugging or to apply the schedule on a former TVM version without Ansor support.
   
   ```python
   >>> print(dag.print_python_code_from_state(state))
   
   i, j, k = tuple(C.op.axis) + tuple(C.op.reduce_axis)
   i_o, i_i = s[C].split(i, factor=16)
   j_o, j_i = s[C].split(j, factor=8)
   s[C].reorder(i_o, j_o, k, j_i, i_i)
   i_o_j_o_fused = s[C].fuse(i_o, j_o)
   ```
   
   We can also replay these steps to get a schedule for `tvm.lower` and `tvm.build`.
   
   ```python
   >>> sche, args = dag.apply_steps_from_state(state)
   >>> print(tvm.lower(sche, args, simple_mode=True))
   
   primfn(A_1: handle, B_1: handle, C_1: handle) -> ()
     attr = {"global_symbol": "main", "tir.noalias": True}
     buffers = {C: Buffer(C_2: handle, float32, [512, 512], []),
                A: Buffer(A_2: handle, float32, [512, 512], []),
                B: Buffer(B_2: handle, float32, [512, 512], [])}
     buffer_map = {A_1: A, B_1: B, C_1: C} {
     for (i.outer.j.outer.fused: int32, 0, 2048) {
       for (j.inner.init: int32, 0, 8) {
         for (i.inner.init: int32, 0, 16) {
           C_2[((((floordiv(i.outer.j.outer.fused, 64)*8192) + (i.inner.init*512)) + (floormod(i.outer.j.outer.fused, 64)*8)) + j.inner.init)] = 0f32
         }
       }
       for (k: int32, 0, 512) {
         for (j.inner: int32, 0, 8) {
           for (i.inner: int32, 0, 16) {
             C_2[((((floordiv(i.outer.j.outer.fused, 64)*8192) + (i.inner*512)) + (floormod(i.outer.j.outer.fused, 64)*8)) + j.inner)] = ((float32*)C_2[((((floordiv(i.outer.j.outer.fused, 64)*8192) + (i.inner*512)) + (floormod(i.outer.j.outer.fused, 64)*8)) + j.inner)]) + ((float32*)A_2[(((floordiv(i.outer.j.outer.fused, 64)*8192) + (i.inner*512)) + k)])*(float32*)B_2[(((k*512) + (floormod(i.outer.j.outer.fused, 64)*8)) + j.inner)])))
           }
         }
       }
     }
   }
   ```
   
   The steps of this state can be serialized into the log file as:
   ```python
   >>> target = tvm.target.create("llvm")
   >>> task = ansor.SearchTask(dag, "test", target)
   >>> inp = ansor.measure.MeasureInput(task, state)
   >>> res = ansor.measure.MeasureResult([0.1], 0, "", 0.2, 1)
   >>> with open("test.log", "w") as fp:
   >>>     ansor.serialization.write_measure_records_to_file(fp.name, [inp], [res])
   
   {"i": [["test", "llvm"], [[], [["SP", 2, 0, 512, [16], 1], ["SP", 2, 2, 512, [8], 1], ["RE", 2, [0, 2, 4, 3, 1]], ["FU", 2, [0, 1]]]]], "r": [[0.1], 0, 0.2, 1], "v": "v0.2"}
   ```
   Ansor serializes all transform steps to the log file; while AutoTVM serializes parameters of a predefined template. The log format discussion would be based on https://discuss.tvm.ai/t/rfc-canonicalizing-autotvm-log-format/7038/.
   
   ---
   
   In the next few PRs, we'll introduce the complete search policy and tutorials for single op/ subgraph schedule search, Relay integration, and tutorials for end-to-end network schedule search, custom rules to support customized search space.
   
   This is a joint work by @merrymercy @jcf94 @minminsun @FrozenGene @comaniac @yangjunpro @yidawang .


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Most comments from the reviewers are addressed.
   
   We summarize the unresolved comments here. They will be addressed by follow-up PRs.
   - The design of `python/tvm/auto_schedule/auto_scheduler.py::auto_schedule`. This is the most important user interface to use the auto-scheduler. We will send an RFC to discuss this.
   - Move `src/auto_schedule/utils.h::ThreadPool` to `src/support/parallle_for.h` (create a new file)
   - Consolidate the usage of `python/tvm/auto_schedule/utils.py::get_const_int`, `python/tvm/auto_schedule/utils.py::get_const_tuple` in the whole TVM code base.


----------------------------------------------------------------
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 pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   > Left a lot of feedback, overall would like to get some clarity in the docs and comments, and think about reducing away some of the concepts to better map to existing TVM systems, it seems like some of the configuration and other bits might be able to fit in nicely with the new machinery from the UnifiedIR. The PR is long and I started to get tired towards the end so I will try to come back for a secondary pass.
   > 
   > Thanks for all your hardwork on this!
   
   Thanks! Your review comments are really helpful!
   I agree that the original intention of our LoopState system is similar to the UnifiedIR to some extent. We also had some periodic discussions with @Hzfengsy during the development. Since the UnifiedIR is not ready for now, we kept the current design, it's great if we can upgrade Ansor with UnifiedIR in the future.
   cc @merrymercy 


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+ * a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+ * total float operation count, consumer/producer relations of each operation stage, whether an
+ * operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+ * to make decisions during search process.
+ * ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+ * (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+ * information got from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;

Review comment:
       This is actually used as the input for `tvm.lower`&`tvm.builder`, currently there seems to be no demand of use input/output respectively.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system

Review comment:
       ```suggestion
    * - transform_step.cc: How each step interact with TIR
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,223 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a schedule for its ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to manipulate the current state.
+The transform history is a sequence of `TransformStep` which will finally be mapped to TVM schedule
+primitives. The steps can also be used for the serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures. The search policy needs to get the
+immediate loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """ A stage in the compute declaration. Similar to tvm.te.schedule.Stage. """
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a list of transformation steps used to construct it.
+
+    Each State corresponds to a specific schedule for its ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The StateObject corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages

Review comment:
       Thanks, use stage cache here is becasue we need to do extra convert from C++ to python befor.
   Since the stage member of state has already been updated to `Array<Stage>` and the stages can be get directly, this cache is no longer needed. I will remove 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=W0703
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=W0703
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose >= 1:

Review comment:
       After checking the code, I find there is actually only 0 and 1.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.

Review comment:
       ```suggestion
       Each State corresponds to a schedule for its ComputeDAG.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*!
+ * \brief The computation information and hardware parameters for a specific schedule search task.
+ */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */
+  ComputeDAG compute_dag;
+  /*! \brief The workload key for target compute declaration. */
+  String workload_key;
+  /*! \brief The target device of this search task. */
+  Target target;
+  /*! \brief The target host device of this search task. */
+  Target target_host;
+  /*! \brief Hardware parameters used in this search task. */
+  HardwareParams hardware_params;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("compute_dag", &compute_dag);
+    v->Visit("workload_key", &workload_key);
+    v->Visit("target", &target);
+    v->Visit("target_host", &target_host);
+    v->Visit("hardware_params", &hardware_params);
+  }
+
+  static constexpr const char* _type_key = "ansor.SearchTask";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SearchTaskNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchTaskNode.
+ * \sa SearchTaskNode
+ */
+class SearchTask : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param compute_dag The ComputeDAG for target compute declaration.
+   * \param workload_key The workload key for target compute declaration.

Review comment:
       Replace target with input

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*!
+ * \brief The computation information and hardware parameters for a specific schedule search task.
+ */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */
+  ComputeDAG compute_dag;
+  /*! \brief The workload key for target compute declaration. */
+  String workload_key;

Review comment:
       remove target 




----------------------------------------------------------------
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] junrushao1994 commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Thank you for the contribution! Just did a round of review on the c++ side. Code is pretty well written and organized - really enjoy it when reading them through. Comments are mostly nitpicks, and feel free to ignore if they don't make 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       Removed from the current code base. cc @tqchen 




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,

Review comment:
       Reduction

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,

Review comment:
       Spatial

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {

Review comment:
       ComputeAtKind

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {

Review comment:
       enum class IteratorKind : int

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {

Review comment:
       StageKind

##########
File path: python/tvm/ansor/record.py
##########
@@ -0,0 +1,157 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file

Review comment:
       record -> measure_record

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op.
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a list of transformation steps used to construct
+ * it.
+ * Each State corresponds to a specific schedule for its ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;

Review comment:
       It would also be useful to think about alternative names, how about concrete?

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op.
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a list of transformation steps used to construct
+ * it.
+ * Each State corresponds to a specific schedule for its ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;

Review comment:
       It would be great if we can have a description about the completeness. e.g. A complete state means that all tile sizes of the state is filled.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       @MarisaKirisame This class inherits the builtin class in python and only sets the `daemon` property to false. Do you have a better name?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+
+    if callable(lookup):
+        args = deserialize_args(workload[1:])
+        return lookup(*args)
+    return lookup
+
+
+@ tvm._ffi.register_func("ansor.workload_key_to_dag")
+def workload_key_to_dag(workload_key):
+    """ Decode a workload key to a compute dag.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    dag : ComputeDAG
+        ComputeDAG to the registered compute declaration.
+    """
+    tensors = workload_key_to_tensors(workload_key)
+    return ComputeDAG(tensors)
+
+
+def make_workload_key_func(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+    args : Args
+        The args of the target function.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the target function.
+    """
+    args = serialize_args(args)
+
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    assert func_name in WORKLOAD_FUNC_REGISTRY, \

Review comment:
       E ... maybe @merrymercy can have some comments on this?




----------------------------------------------------------------
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] MarisaKirisame commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   I had been looking over this PR a bit more, and it seems like a lot of review is dropped when file is updated. This is no one flaw's but a flaw in github's review design, and I think it just mean we should be careful in the upcoming Ansor review - just because code is merged, doesnt mean it is at top quality, and we should continue going over 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=W0703
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=W0703
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose >= 1:

Review comment:
       Then there are two directions I suggested:
   1. Comment on verbose saying it could be either 0 or 1, and make sure all use cases are `verbose==1` instead of `verbose>=1`.
   2. Remove `verbose` and use dmlc logging system.
   




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.

Review comment:
       ```suggestion
       the init state of input ComputeDAG.
   ```
   Remove all "target" before "ComputeDAG", "compute dag", "compute decleration", "SearchTask", "search task", "State", "Stage" in all files. They are redundant.




----------------------------------------------------------------
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] FrozenGene commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters

Review comment:
       I prefer we setting the default value rather than user specify it. The value is related with hardware more directly too (avx2 / avx512 / neon decides the max factor naturelly we should vectorize). The default value of 32 should work well on intel / arm.




----------------------------------------------------------------
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] jroesch commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   @jcf94 and @merrymercy thanks for all the hard work! Can I request that we put another unresolved issue? In my opinion the written English parts i.e comments, explanations, etc could still use some improvement with both content and grammar and I would propose in general that we do some at least 1 or 2 rounds of full documentation polish (comments, examples, tests, tutorials, etc)  before we officially release a feature (in this case when all of Ansor is landed in master). We tried to do this with Relay but I think we should continue to strive to do a better job with new features like this. 


----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {
+      res = Iterator(name, Range::FromMinExtent(tosplit_min, l), it->iter_type, kNone);
+      tosplit_min = 0;
+      tosplit_extent = indexdiv(tosplit_extent + l - 1, l);
+    } else {
+      res = Iterator(name, Range(), it->iter_type, kNone);
+      tosplit_min = tosplit_extent = PrimExpr();
+    }
+    outs.push_back(std::move(res));
+  }
+
+  Range range;
+  if (tosplit_min.defined() && tosplit_extent.defined()) {
+    range = Range::FromMinExtent(tosplit_min, tosplit_extent);
+  }
+  if (inner_to_outer) {
+    outs.push_back(Iterator(it->name + ".0", range, it->iter_type, kNone));
+    // Reverse the Iterator array
+    Array<Iterator> temp(outs.rbegin(), outs.rend());
+    outs = std::move(temp);
+  } else {
+    outs.push_back(
+        Iterator(it->name + "." + std::to_string(lengths.size()), range, it->iter_type, kNone));
+  }
+
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(), stage->iters.begin() + iter_id);
+  new_iters.insert(new_iters.end(), outs.begin(), outs.end());
+  new_iters.insert(new_iters.end(), stage->iters.begin() + iter_id + 1, stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return outs;
+}
+
+Array<Iterator> State::DoSplitStep(const SplitStep& step) {
+  return DoSplitStepCommon(step->stage_id, step->iter_id, step->lengths, step->inner_to_outer);
+}
+
+Iterator State::DoFuseStep(const FuseStep& step) {
+  int stage_id = step->stage_id;
+  const Stage& stage = operator->()->stages[stage_id];
+
+  String new_name;
+  PrimExpr new_extent = 1;
+  IteratorType new_iter_type = kSpecial;
+
+  for (size_t i = 0; i < step->fused_ids.size(); ++i) {
+    if (i > 0) {
+      CHECK_EQ(step->fused_ids[i]->value, step->fused_ids[i - 1]->value + 1);
+    }
+
+    const Iterator& it = stage->iters[step->fused_ids[i]];
+    new_name = new_name + it->name + "@";
+
+    if (it->range.defined() && new_extent.defined()) {
+      new_extent = new_extent * it->range->extent;
+    } else {
+      new_extent = PrimExpr();
+    }
+
+    if (i == 0) {
+      new_iter_type = it->iter_type;
+    } else {
+      if (new_iter_type != it->iter_type) {
+        new_iter_type = kMixed;
+      }
+    }
+  }
+
+  Range range;
+  if (new_extent.defined()) {
+    range = Range::FromMinExtent(0, new_extent);
+  }
+  Iterator new_it = Iterator(new_name, range, new_iter_type, kNone);
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(),
+                   stage->iters.begin() + step->fused_ids.front());
+  new_iters.push_back(new_it);
+  new_iters.insert(new_iters.end(), stage->iters.begin() + step->fused_ids.back() + 1,
+                   stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return new_it;
+}
+
+void State::DoSteps(const ComputeDAG& dag) {
+  CHECK(operator->()->stages.size()) << "Invalid State with empty operation stages.";
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  for (const auto& step : operator->()->transform_steps) {
+    if (complete_rate >= 0 && ct++ > operator->()->transform_steps.size() * complete_rate) {
+      break;
+    }
+    if (auto ps = step.as<ReorderStepNode>()) {
+      DoReorderStep(GetRef<ReorderStep>(ps));
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      DoSplitStep(GetRef<SplitStep>(ps));
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      DoFuseStep(GetRef<FuseStep>(ps));
+    } else {
+      LOG(FATAL) << "Invalid step: " << step;
+    }
+  }
+}
+
+// Print stage to ostream
+void PrintStage(std::ostream* os, int stage_id, const StateNode* state, size_t base_indent,
+                bool delete_trivial_loop) {
+  const Stage& stage = state->stages[stage_id];
+
+  if (stage->attrs.auto_unroll_max_step != 0) {
+    for (size_t j = 0; j < base_indent; ++j) {
+      *os << " ";
+    }
+    *os << stage->op->name << " auto_unroll: " << stage->attrs.auto_unroll_max_step << "\n";
+  }
+  if (stage->attrs.storage_offset != 0) {
+    for (size_t j = 0; j < base_indent; ++j) {
+      *os << " ";
+    }
+    *os << stage->op->name << " storage_offset: " << stage->attrs.storage_offset << "\n";
+  }
+
+  size_t indent = 0;
+  for (size_t i = 0; i < stage->iters.size(); ++i) {
+    const Iterator& iter = stage->iters[i];
+
+    if (!(delete_trivial_loop && iter->range.defined() && is_one(iter->range->extent))) {
+      for (size_t j = 0; j < base_indent + indent; ++j) {
+        *os << " ";
+      }
+      switch (iter->annotation) {

Review comment:
       Let's use an array of names instead. Although effectively there are the same, but might look more clear.




----------------------------------------------------------------
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] FrozenGene commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: tests/python/unittest/test_ansor_measure.py
##########
@@ -0,0 +1,67 @@
+# 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.
+
+"""Test measurement and log serialization"""
+
+import tvm
+from tvm import ansor
+import tempfile
+
+from test_ansor_common import get_tiled_matmul
+
+
+def test_serialization():
+    dag, s = get_tiled_matmul()
+    target = tvm.target.create("llvm")

Review comment:
       we should add `if not tvm.runtime.enabled("llvm"):` to check the environment has enabled `llvm`. Other targets should be done 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] MarisaKirisame commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/__init__.py
##########
@@ -0,0 +1,34 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       should this folder be named auto_scheduler?




----------------------------------------------------------------
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] yangjunpro commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.

Review comment:
       +1

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.

Review comment:
       Seconds, it is actually a wrapper of the _multiprocessing.Process.join()_ method.
   Better to make it more specific in the comments. 

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,466 @@
+# 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.
+
+"""
+Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling.
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# We use fork and a global variable to copy arguments between processings.
+# This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The SearchTask of this measure.
+    state : State
+        The State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbose : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbose : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        The number of times to run the generated code for taking average.
+        We call these runs as one `repeat` of measurement.
+    repeat : int = 1
+        The number of times to repeat the measurement.
+        In total, the generated code will be run (1 + number x repeat) times,
+        where the first "1" is warm up and will be discarded.
+        The returned result contains `repeat` costs,
+        each of which is an average of `number` costs.
+    min_repeat_ms : int = 0
+        The minimum duration of one `repeat` in milliseconds.
+        By default, one `repeat` contains `number` runs. If this parameter is set,
+        the parameters `number` will be dynamically adjusted to meet the
+        minimum duration requirement of one `repeat`.
+        i.e., When the run time of one `repeat` falls below this time, the `number` parameter
+        will be automatically increased.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \

Review comment:
       What is the purpose to limit the length of the error message?

##########
File path: python/tvm/ansor/record.py
##########
@@ -0,0 +1,157 @@
+# 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.
+
+""" Serialization and other I/O support for tuning logs (measurement records). """
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=None, skip_lines=None):

Review comment:
       Why not specify the _skip_line_ with _0_ as default value rather than _None_? Thus we can avoid a following condition check.
   So as _max_lines_.

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       Maybe here what we mean is a synchronous pool rather than an asynchronous one? 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/__init__.py
##########
@@ -0,0 +1,34 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       I think @MarisaKirisame means the namespace should be a noun. So auto_scheduler is better. I think so




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.

Review comment:
       ```suggestion
           Apply the history transform steps from a State to get a TVM 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jcf94 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,162 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_by_func(func):

Review comment:
       Renamed to `register_workload`.




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {

Review comment:
       Let us change the namespace to `auto_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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);

Review comment:
       @jcf94 you are right indeed. I did a grep and saw only very few tvm objects are registered using OBJECT_TYPE, which made me leave this comment. Do you prefer it if we keep this consistency through TVM codebase?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/utils.cc
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;

Review comment:
       cc @merrymercy 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):

Review comment:
       Updated the class name `Builder` -> `ProgramBuilder` & `Runner` -> `ProgramRunner`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):

Review comment:
       This follows the name convention in autotvm.
   We separate the measurement into two steps: build and run.
   So we have the flow `MeasureInput -> BuildResult -> MeasureResult`
   A builder takes `MeasureInput` and returns `BuildResult`. A runner then takes `BuildResult` and returns `MeasureResult`




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.

Review comment:
       ```suggestion
         The number of schedules to be measured at each search round.
         The whole schedule search process will try a total number of `num_measure_trials` in several rounds.
   ```
   
   Be consistent with programs, schedules, and states.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ApplySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class of transformation steps. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */

Review comment:
       Remove all `target` in this file




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]

Review comment:
       Good point, that dose seem better.




----------------------------------------------------------------
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] tqchen commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   I made a change request about the namespace change to `auto_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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {
+      res = Iterator(name, Range::FromMinExtent(tosplit_min, l), it->iter_type, kNone);
+      tosplit_min = 0;
+      tosplit_extent = indexdiv(tosplit_extent + l - 1, l);
+    } else {
+      res = Iterator(name, Range(), it->iter_type, kNone);
+      tosplit_min = tosplit_extent = PrimExpr();
+    }
+    outs.push_back(std::move(res));
+  }
+
+  Range range;
+  if (tosplit_min.defined() && tosplit_extent.defined()) {
+    range = Range::FromMinExtent(tosplit_min, tosplit_extent);
+  }
+  if (inner_to_outer) {
+    outs.push_back(Iterator(it->name + ".0", range, it->iter_type, kNone));
+    // Reverse the Iterator array
+    Array<Iterator> temp(outs.rbegin(), outs.rend());
+    outs = std::move(temp);
+  } else {
+    outs.push_back(
+        Iterator(it->name + "." + std::to_string(lengths.size()), range, it->iter_type, kNone));
+  }
+
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(), stage->iters.begin() + iter_id);
+  new_iters.insert(new_iters.end(), outs.begin(), outs.end());
+  new_iters.insert(new_iters.end(), stage->iters.begin() + iter_id + 1, stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return outs;
+}
+
+Array<Iterator> State::DoSplitStep(const SplitStep& step) {
+  return DoSplitStepCommon(step->stage_id, step->iter_id, step->lengths, step->inner_to_outer);
+}
+
+Iterator State::DoFuseStep(const FuseStep& step) {
+  int stage_id = step->stage_id;
+  const Stage& stage = operator->()->stages[stage_id];
+
+  String new_name;
+  PrimExpr new_extent = 1;
+  IteratorType new_iter_type = kSpecial;
+
+  for (size_t i = 0; i < step->fused_ids.size(); ++i) {
+    if (i > 0) {
+      CHECK_EQ(step->fused_ids[i]->value, step->fused_ids[i - 1]->value + 1);
+    }
+
+    const Iterator& it = stage->iters[step->fused_ids[i]];
+    new_name = new_name + it->name + "@";
+
+    if (it->range.defined() && new_extent.defined()) {
+      new_extent = new_extent * it->range->extent;
+    } else {
+      new_extent = PrimExpr();
+    }
+
+    if (i == 0) {
+      new_iter_type = it->iter_type;
+    } else {
+      if (new_iter_type != it->iter_type) {
+        new_iter_type = kMixed;
+      }
+    }
+  }
+
+  Range range;
+  if (new_extent.defined()) {
+    range = Range::FromMinExtent(0, new_extent);
+  }
+  Iterator new_it = Iterator(new_name, range, new_iter_type, kNone);
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(),
+                   stage->iters.begin() + step->fused_ids.front());
+  new_iters.push_back(new_it);
+  new_iters.insert(new_iters.end(), stage->iters.begin() + step->fused_ids.back() + 1,
+                   stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return new_it;
+}
+
+void State::DoSteps(const ComputeDAG& dag) {
+  CHECK(operator->()->stages.size()) << "Invalid State with empty operation stages.";
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  for (const auto& step : operator->()->transform_steps) {
+    if (complete_rate >= 0 && ct++ > operator->()->transform_steps.size() * complete_rate) {
+      break;
+    }
+    if (auto ps = step.as<ReorderStepNode>()) {
+      DoReorderStep(GetRef<ReorderStep>(ps));
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      DoSplitStep(GetRef<SplitStep>(ps));
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      DoFuseStep(GetRef<FuseStep>(ps));
+    } else {
+      LOG(FATAL) << "Invalid step: " << step;
+    }
+  }
+}
+
+// Print stage to ostream
+void PrintStage(std::ostream* os, int stage_id, const StateNode* state, size_t base_indent,
+                bool delete_trivial_loop) {
+  const Stage& stage = state->stages[stage_id];
+
+  if (stage->attrs.auto_unroll_max_step != 0) {
+    for (size_t j = 0; j < base_indent; ++j) {
+      *os << " ";
+    }
+    *os << stage->op->name << " auto_unroll: " << stage->attrs.auto_unroll_max_step << "\n";
+  }
+  if (stage->attrs.storage_offset != 0) {
+    for (size_t j = 0; j < base_indent; ++j) {
+      *os << " ";
+    }
+    *os << stage->op->name << " storage_offset: " << stage->attrs.storage_offset << "\n";
+  }
+
+  size_t indent = 0;
+  for (size_t i = 0; i < stage->iters.size(); ++i) {
+    const Iterator& iter = stage->iters[i];
+
+    if (!(delete_trivial_loop && iter->range.defined() && is_one(iter->range->extent))) {
+      for (size_t j = 0; j < base_indent + indent; ++j) {
+        *os << " ";
+      }
+      switch (iter->annotation) {

Review comment:
       Let's use an array of names instead to avoid switch. Although effectively they are the same, but might look more clear.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
+1. We want fast incremental change to the loop structures;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+After the search is done, we will lower this IR to TVM IR with TVM's schedule primitives.
+Because we share a lot common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")

Review comment:
       Yes, the State is used by SearchPolicy and the implementations are all in C++. Here we have Python APIs for State operations is to prepare for providing custom search rule in the future.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover auto schedule generation for different
+ * ops/subgraphs, and in the meantime, a custom rule mechanism will be provided to enable
+ * user-defined template search. (which should play a same role as the current AutoTVM template)
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search and returns the best schedule get.
+ * 2. Imformations about the target ops/subgraphs can be acquired from SearchTask, this structure
+ * also contains HardwareParams which can be used to limit the search space. (For exp. limit the
+ * max vectorize size depending on the vector unit weight of a specific device)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to SearchPolicyNode.
+   */
+  virtual void Callback(SearchPolicyNode* policy) = 0;
+
+  static constexpr const char* _type_key = "ansor.SearchCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchCallbackNode.
+ * \sa SearchCallbackNode
+ */
+class SearchCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(SearchCallback, ObjectRef, SearchCallbackNode);
+};
+
+/*!
+ * \brief The base class for search policy.
+ */
+class SearchPolicyNode : public Object {
+ public:
+  /*! \brief The current search task. */
+  SearchTask cur_task;
+  /*!
+   * \brief Verbose level to control the screen output during schedule search.
+   * 0 for silent, 1 to output information.
+   */
+  int verbose;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("cur_task", &cur_task);
+    v->Visit("verbose", &verbose);
+  }
+
+  /*!
+   * \brief Do schedule search for a task. Takes the SearchTask as input and returns the best state
+   * get during the search process.
+   * \param task The target search task.
+   * \param num_measure_trials Total schedules to be tried during this search.
+   * \param early_stopping Early stop if no better schedule is found.
+   * \param num_measures_per_round Max measure batch in one search round.
+   * \param verbose Verbose level. 0 for silent, 1 to output information during schedule search.
+   * \param measurer A ProgramMeasurer which packs Builder & Runner inside.
+   * \param pre_search_callbacks SearchCallback to be called before schedule search.
+   * \return The best state get.
+   */
+  virtual State Search(SearchTask task, int num_measure_trials, int early_stopping,
+                       int num_measures_per_round, int verbose, ProgramMeasurer measurer,
+                       Array<SearchCallback> pre_search_callbacks) = 0;
+
+  /*!
+   * \brief Call SearchCallback with the current SearchPolicyNode
+   * \param callbacks SearchCallback to be called.
+   */
+  void RunCallbacks(const Array<SearchCallback>& callbacks);
+
+  static constexpr const char* _type_key = "ansor.SearchPolicy";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchPolicyNode, Object);
+
+ protected:
+  /*!
+   * \brief The set of already measured states.
+   * We store the string format for redundancy check.
+   */
+  std::unordered_set<String> measured_states_set_;
+  /*! \brief The array of already measured states. */
+  std::vector<State> measured_states_vector_;
+  /*! \brief The throughputs of already measured states */
+  std::vector<float> measured_states_throughputs_;
+};
+
+/*!
+ * \brief Managed reference to SearchPolicyNode.

Review comment:
       This is to keep the same with other TVM class definitions: `Managed reference class to xxxNode.`.




----------------------------------------------------------------
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] MarisaKirisame commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   You guys used crossover, and it often suck in genetic algorithm. Had you guys done ablation study of genetic algorithm with pure-mutation approach?


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The generation function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__

Review comment:
       Great suggestion! Updated the implementation with reference to `tvm._ffi.register_func`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state.
+
+        The states can lose complete bound information after some transform steps
+        (e.g., compute_at).
+        We can call this function to infer and fill all the bound information.
+        This function calls TVM InferBound pass internally to get the bound.
+        The returned state of this function is guaranteed to have complete iterator extent
+        information.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        state : State
+            The State with complete bound information.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return State(_ffi_api.ComputeDAGInferBoundFromState(self, state_obj), self)
+
+    def __hash__(self):
+        # TODO(merrymercy): Implement this more carefully and move this to c++ as a member function

Review comment:
       Some c++ functions may need this. Moving it to c++ makes it more convenient for c++ callers.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,

Review comment:
       `filename if filename else ""`  and `error_msg if error_msg else ""` seems meaningless. If both of them could be `None` then please update the docstring.
   
   ditto to `MeasureResult`

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=W0703
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=W0703
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose >= 1:
+            if error_no == MeasureErrorNo.NO_ERROR:
+                print(".", end="")
+            else:
+                print(".E", end="")  # Build error
+        return filename, args, error_no, error_msg, time.time() - tic
+
+    res = call_func_with_timeout(timeout, timed_func)
+    if isinstance(res, TimeoutError):
+        if verbose >= 1:
+            print(".T", end="")  # Build timeout
+        res = None, [], MeasureErrorNo.BUILD_TIMEOUT, None, timeout
+
+    return res
+
+
+@tvm._ffi.register_func("ansor.local_builder.build")
+def local_builder_build(inputs, timeout, n_parallel, build_func, verbose):
+    """ Local builder build function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    global GLOBAL_BUILD_ARGUMENTS
+    GLOBAL_BUILD_ARGUMENTS = (inputs, build_func, timeout, verbose)
+
+    pool = NoDaemonPool(n_parallel)
+    tuple_res = pool.map(local_build_worker, range(len(inputs)))
+    pool.terminate()
+    pool.join()
+    del pool
+
+    results = []
+    for res in tuple_res:
+        results.append(BuildResult(*res))
+
+    return results
+
+@tvm._ffi.register_func("ansor.local_runner.run")
+def local_run(inputs, build_results, timeout, number, repeat, min_repeat_ms, cooldown_interval,
+              verbose):
+    """ Local runner run function. """
+    max_float = 1e10  # We use 1e10 instead of sys.float_info.max for better readability in log
+
+    def timed_func(inp, build_res):
+        tic = time.time()
+        error_no = 0
+        error_msg = None
+        try:
+            func = module.load_module(build_res.filename)
+            ctx = ndarray.context(str(inp.task.target), 0)
+            time_f = func.time_evaluator(
+                func.entry_name, ctx, number=number, repeat=repeat, min_repeat_ms=min_repeat_ms)
+        # pylint: disable=W0703
+        except Exception:
+            costs = (max_float,)
+            error_no = MeasureErrorNo.COMPILE_DEVICE
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            try:
+                args = [ndarray.empty(get_const_tuple(x.shape), x.dtype, ctx) for x in
+                        build_res.args]
+                ctx.sync()
+

Review comment:
       remove this line.

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""

Review comment:
       From the functionality of this file, it seems improper to name it `serialization.py`. Maybe using `record.py` as AutoTVM or we could figure out a better name.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=W0703

Review comment:
       Better to use `disable=broad-except` to make it more clear.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):

Review comment:
       The naming of this function is a bit of confusing. We are actually registering a graph that produces the buffer instead of just the buffer (tensor) itself. The logic is almost the same as `register_workload_func`. The only differences are 1) registering a function vs. a list of tensors, and 2) function name as the workload key vs. hashed DAG as the key. IMHO, `register_func_workload` and `register_tensor_workload` are better, but any names that could address above concerns would be good.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.

Review comment:
       Please correct me if I was wrong. It seems to me that this function is not "decoding" a workload key but just looking up for the corresponding registered workload?

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,157 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    psutil = None
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.
+    This allows us to start new processings inside the worker function"""
+
+    def __init__(self, *args, **kwargs):
+        kwargs['context'] = NoDaemonContext()
+        super().__init__(*args, **kwargs)
+
+    def __reduce__(self):
+        pass
+
+
+def kill_child_processes(parent_pid, sig=signal.SIGTERM):
+    """kill all child processes recursively"""
+    try:
+        parent = psutil.Process(parent_pid)

Review comment:
       From the top of this file it seems possible for `psutil` to be None. Should we check that in this function?

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_size=-1, skip_size=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_size : Int
+            The maximum number of lines. -1 means read all lines.
+        skip_size : Int
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(
+            self, max_size, skip_size)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename: str):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def write_measure_records_to_file(filename, inputs, results):
+    """
+    Write(append) measure records to file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.
+    results: List[MeasureResults]
+        The target MeasureResults to be written.
+    """
+    _ffi_api.WriteMeasureRecordsToFile(filename, inputs, results)
+
+def best_measure_pair_in_file(filename, workload_key=None, target=None):
+    """ Return the best measurement pair form a log file

Review comment:
       Maybe we should also mention that it's possible to return an invalid pair if all records in the log file are invalid.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str

Review comment:
       Please use `str`, `int`, `float`. Ditto to the rest functions in this file.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None

Review comment:
       Better to move them to the top of this file.

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):

Review comment:
       Maybe we should have a file for callback classes?

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS

Review comment:
       Should test if GLOBAL_BUILD_ARGUMENTS is None or not.

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_size=-1, skip_size=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_size : Int
+            The maximum number of lines. -1 means read all lines.
+        skip_size : Int
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(
+            self, max_size, skip_size)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename: str):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def write_measure_records_to_file(filename, inputs, results):
+    """
+    Write(append) measure records to file.

Review comment:
       How about just calling it `append_measure_records_to_file`?

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_size=-1, skip_size=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_size : Int

Review comment:
       - `max_line`, `skip_line` might be clearer.
   - s/Int/int/

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_size=-1, skip_size=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_size : Int
+            The maximum number of lines. -1 means read all lines.
+        skip_size : Int
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(
+            self, max_size, skip_size)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename: str):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def write_measure_records_to_file(filename, inputs, results):
+    """
+    Write(append) measure records to file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.
+    results: List[MeasureResults]
+        The target MeasureResults to be written.
+    """
+    _ffi_api.WriteMeasureRecordsToFile(filename, inputs, results)
+
+def best_measure_pair_in_file(filename, workload_key=None, target=None):
+    """ Return the best measurement pair form a log file
+
+    Parameters
+    ----------
+    filename : Str
+        File name to load log from.
+    workload_key : Str

Review comment:
       - s/Str/Optional[str] for `workload_key` and 'target'.
   - Explain the behavior when `workload_key` or `target` is None.
   

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+
+    if callable(lookup):
+        args = deserialize_args(workload[1:])
+        return lookup(*args)
+    return lookup
+
+
+@ tvm._ffi.register_func("ansor.workload_key_to_dag")
+def workload_key_to_dag(workload_key):
+    """ Decode a workload key to a compute dag.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    dag : ComputeDAG
+        ComputeDAG to the registered compute declaration.
+    """
+    tensors = workload_key_to_tensors(workload_key)
+    return ComputeDAG(tensors)
+
+
+def make_workload_key_func(func, args):

Review comment:
       For both `make_workload_key_XX` functions.
   - What's the purpose of these functions?
   - The same problem to the naming.
   - It looks like we can reuse these 2 functions in `register_workload_XX` to maintain the key generation in a single place.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):

Review comment:
       `hash_compute_dag` might be better as "hash" here could be a verb.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+

Review comment:
       Those 3 functions should be in util.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+
+    if callable(lookup):

Review comment:
       Add comment to explain this is the case of registering a function; otherwise `lookup` would be a list of tensors.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):

Review comment:
       Should verbose default be 0?
   Ditto to `Runner`.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+
+    if callable(lookup):
+        args = deserialize_args(workload[1:])
+        return lookup(*args)
+    return lookup
+
+
+@ tvm._ffi.register_func("ansor.workload_key_to_dag")
+def workload_key_to_dag(workload_key):
+    """ Decode a workload key to a compute dag.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    dag : ComputeDAG
+        ComputeDAG to the registered compute declaration.
+    """
+    tensors = workload_key_to_tensors(workload_key)
+    return ComputeDAG(tensors)
+
+
+def make_workload_key_func(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+    args : Args
+        The args of the target function.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the target function.
+    """
+    args = serialize_args(args)
+
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    assert func_name in WORKLOAD_FUNC_REGISTRY, \

Review comment:
       Why we need this check in this function?

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_size=-1, skip_size=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_size : Int
+            The maximum number of lines. -1 means read all lines.
+        skip_size : Int
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(
+            self, max_size, skip_size)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename: str):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def write_measure_records_to_file(filename, inputs, results):
+    """
+    Write(append) measure records to file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.
+    results: List[MeasureResults]
+        The target MeasureResults to be written.
+    """
+    _ffi_api.WriteMeasureRecordsToFile(filename, inputs, results)
+
+def best_measure_pair_in_file(filename, workload_key=None, target=None):
+    """ Return the best measurement pair form a log file
+
+    Parameters
+    ----------
+    filename : Str
+        File name to load log from.
+    workload_key : Str
+        The workload key of the target compute declaration.
+    target : Str
+        The target device.
+
+    Returns
+    -------
+    input : MeasureInput
+        The best State's MeasureInput from this log fine.
+    result : MeasureResult
+        The best State's MeasureResult from this log fine.
+    """
+    log_reader = LogReader(filename)
+    best_cost = 1e30
+    best_inp = None
+    best_res = None
+
+    for inp, res in log_reader:
+        if res.error_no != MeasureErrorNo.NO_ERROR:
+            continue
+        if workload_key and inp.task.workload_key != workload_key:
+            continue
+        if target and inp.task.target.target_name != target.target_name:
+            continue
+
+        costs = []
+        for value in res.costs:
+            costs.append(value.value)

Review comment:
       `costs = [v.value for v in res.costs]` would be more concise.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]

Review comment:
       What's the expected type of `workload`?

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : Str
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_size=-1, skip_size=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_size : Int
+            The maximum number of lines. -1 means read all lines.
+        skip_size : Int
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(
+            self, max_size, skip_size)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename: str):

Review comment:
       Be consistent with the type annotation.




----------------------------------------------------------------
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] yangjunpro commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters

Review comment:
       Yes, sometimes we may choose not to use all the cores available in the host system since some of the cores need to be kept for other purpose, such as data pre-processing, etc. So I think it's better to leave _num_cores_  a user-specified options with default value. 
   Regarding to _vector_unit_bytes_/_cache_line_bytes_, in my understanding they are  primitive hardware configurations, maybe we can consider to auto-detect it? 
   The principle is that we should do auto-detection as much as we can, without too much overkill. 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {
+      res = Iterator(name, Range::FromMinExtent(tosplit_min, l), it->iter_type, kNone);
+      tosplit_min = 0;
+      tosplit_extent = indexdiv(tosplit_extent + l - 1, l);
+    } else {
+      res = Iterator(name, Range(), it->iter_type, kNone);
+      tosplit_min = tosplit_extent = PrimExpr();
+    }
+    outs.push_back(std::move(res));
+  }
+
+  Range range;
+  if (tosplit_min.defined() && tosplit_extent.defined()) {
+    range = Range::FromMinExtent(tosplit_min, tosplit_extent);
+  }
+  if (inner_to_outer) {
+    outs.push_back(Iterator(it->name + ".0", range, it->iter_type, kNone));
+    // Reverse the Iterator array
+    Array<Iterator> temp(outs.rbegin(), outs.rend());
+    outs = std::move(temp);
+  } else {
+    outs.push_back(
+        Iterator(it->name + "." + std::to_string(lengths.size()), range, it->iter_type, kNone));
+  }
+
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(), stage->iters.begin() + iter_id);
+  new_iters.insert(new_iters.end(), outs.begin(), outs.end());
+  new_iters.insert(new_iters.end(), stage->iters.begin() + iter_id + 1, stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return outs;
+}
+
+Array<Iterator> State::DoSplitStep(const SplitStep& step) {
+  return DoSplitStepCommon(step->stage_id, step->iter_id, step->lengths, step->inner_to_outer);
+}
+
+Iterator State::DoFuseStep(const FuseStep& step) {
+  int stage_id = step->stage_id;
+  const Stage& stage = operator->()->stages[stage_id];
+
+  String new_name;
+  PrimExpr new_extent = 1;
+  IteratorType new_iter_type = kSpecial;
+
+  for (size_t i = 0; i < step->fused_ids.size(); ++i) {
+    if (i > 0) {
+      CHECK_EQ(step->fused_ids[i]->value, step->fused_ids[i - 1]->value + 1);
+    }
+
+    const Iterator& it = stage->iters[step->fused_ids[i]];
+    new_name = new_name + it->name + "@";
+
+    if (it->range.defined() && new_extent.defined()) {
+      new_extent = new_extent * it->range->extent;
+    } else {
+      new_extent = PrimExpr();
+    }
+
+    if (i == 0) {
+      new_iter_type = it->iter_type;
+    } else {
+      if (new_iter_type != it->iter_type) {
+        new_iter_type = kMixed;
+      }
+    }
+  }
+
+  Range range;
+  if (new_extent.defined()) {
+    range = Range::FromMinExtent(0, new_extent);
+  }
+  Iterator new_it = Iterator(new_name, range, new_iter_type, kNone);
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(),
+                   stage->iters.begin() + step->fused_ids.front());
+  new_iters.push_back(new_it);
+  new_iters.insert(new_iters.end(), stage->iters.begin() + step->fused_ids.back() + 1,
+                   stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return new_it;
+}
+
+void State::DoSteps(const ComputeDAG& dag) {

Review comment:
       The dag is left for some future steps that may change the op stages(e.g. CacheRead/CacheWrite), will add more doc on 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.

Review comment:
       ```suggestion
           The ComputeDAG for the compute declaration.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target)

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).

Review comment:
       ```suggestion
       We convert a compute declaration described by `tvm.compute` (could be a single operator or a
       subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
       a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the total float operation count, consumer/producer relations of each operation 
       stage, whether an operation stage should be tiled/compute inlined ...). These analyses can
       help the search policy to make decisions during search process.
   
       ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
       (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
       information got from TVM schedule ...).
   ```

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.

Review comment:
       ```suggestion
           Apply the history transform steps of a State to  get a TVM schedule.
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.

Review comment:
       ```suggestion
           The workload key for the compute declaration.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target)

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""

Review comment:
       ```suggestion
   The definition of the "state" in search.
   
   Each LoopState corresponds to a schedule for its ComputeDAG.
   A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
   construct the loop structure.
   The loop structure keeps a preview of how the schedule will finally look like after lowering the
   current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
   During the schedule search process, the loop structure can provide search policy with necessary
   information on how to manipulate the current state.
   The transform history is a sequence of `TransformStep` which will finally be mapped to TVM schedule
   primitives. The steps can also be used for the serialization of a state.
   
   The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
   We don't use the existing TVM IR but to extend a new structure on it is because:
   1. We want fast incremental change to the loop structures. The search policy needs to get the immediate
   loop structures update rather than after TVM lowering;
   2. We want serializable transform history for replay, backtracking, and mutation;
   3. We may create some macro schedule primitives that represent the combination of several
   TVM schedule primitives.
   
   When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
   Since we share a lot of common objects during search, the transformation is implemented in
   copy on write style. All objects are immutable, which is similar to TVM IR.
   """
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages
+        self.stage_id_map = {}    # A dict maps operation to stage id
+        self._update_stage_id_map()
+
+    @property
+    def stages(self):
+        """
+        Returns
+        -------
+        stages : List[Stage]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return self.stages_cache
+
+    @property
+    def stage_ops(self):
+        """
+        Returns
+        -------
+        ops: List[Operation]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return [stage.op for stage in self.stages_cache]
+
+    def reorder(self, stage, order):
+        """ Schedule primitive corresponds to te.reorder.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be reordered, can be a Stage order index, Stage operation or stage

Review comment:
       ```suggestion
               The Stage to be reordered, which can be a Stage order index, Stage operation or stage
   ```
   Do not overuse `target`. It already has its own meaning in `compute_at` (i.e., the target of compute_at`)

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.

Review comment:
       ```suggestion
       the init state of input ComputeDAG.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target)

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages
+        self.stage_id_map = {}    # A dict maps operation to stage id
+        self._update_stage_id_map()
+
+    @property
+    def stages(self):
+        """
+        Returns
+        -------
+        stages : List[Stage]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return self.stages_cache
+
+    @property
+    def stage_ops(self):
+        """
+        Returns
+        -------
+        ops: List[Operation]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return [stage.op for stage in self.stages_cache]
+
+    def reorder(self, stage, order):
+        """ Schedule primitive corresponds to te.reorder.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be reordered, can be a Stage order index, Stage operation or stage
+            output tensor.
+        order : List[Iterator]
+            Iterators in the expected order
+        """
+        stage_id = self._resolve_stage_id(stage)
+
+        self.state_object = _ffi_api.StateReorder(self.state_object, stage_id, order)
+        self._clear_cache()
+
+    def split(self, stage, iterator, lengths, inner_to_outer=True):
+        """ Schedule primitive corresponds to te.split.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be split, can be a Stage order index, Stage operation or stage

Review comment:
       ```suggestion
               The Stage to be split, can be a Stage order index, Stage operation or stage
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.

Review comment:
       ```suggestion
         Callback functions called after each measurement.
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.

Review comment:
       ```suggestion
         Stop the tuning early if getting no improvement after n measurements.
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.

Review comment:
       ```suggestion
         The number of schedules to be measured at each search round.
         The whole schedule search process is designed to  try a total number of 
         `num_measure_trials` in several rounds.
   ```
   
   Be consistent with programs, schedules, and states.

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.

Review comment:
       ```suggestion
         The number of measurement trials.
         The search policy measures `num_measure_trials` schedules in total and returns the best one among them.
         With `num_measure_trials` == 0, the policy will do the schedule search but won't involve measurement.
         This can be used to get a runnable schedule quickly without auto-tuning.
   ```
   
   Keep consistent with `states`, `schedules` 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The target search task or workload key.

Review comment:
       ```suggestion
           The search task or workload key.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target).

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.

Review comment:
       ```suggestion
               The state from which we get transform steps
   ```

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state using TVM schedule.

Review comment:
       ```suggestion
           Infer and fill the bound of all iterators of a state.
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The target search task or workload key.
+    target : tvm.target.Target
+        The target device of this schedule search.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str] = 'default'
+        The search policy to be used for schedule search.
+    hardware_params : Optional[HardwareParams]
+        The hardware parameters of this schedule search.
+    tuning_options : Optional[TuningOptions]
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`

Review comment:
       ```suggestion
           A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`
   ```
   Do not overuse target because target already has its meaning (i.e. the hardware target)

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state using TVM schedule.
+
+        State api supports to define a split step with its split factor to be a blank placeholder,
+        so sometimes we may get a State will incomplete iterator extent information.
+        And another situation is after some steps (for exp. compute_at), it may be hard to track
+        the extent change of all iterators.
+
+        We perform infer bound using TVM schedule and fill the State with those information. After
+        applying this methods, the State is guaranteed to have complete interator extent
+        information.

Review comment:
       ```suggestion
           The states can lose complete bound information after some transform steps (e.g., compute_at).
           We can call this function to infer and fill all the bound information.
           This function calls TVM InferBound pass internally to get the bound.
   
           The returned state of this function is guaranteed to have complete iterator extent
           information.
   ```
   
   Is `for exp` correct? I saw you use it frequently but I don't think it is a correct idiom.  Please use `e.g.,`.

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state using TVM schedule.
+
+        State api supports to define a split step with its split factor to be a blank placeholder,
+        so sometimes we may get a State will incomplete iterator extent information.
+        And another situation is after some steps (for exp. compute_at), it may be hard to track
+        the extent change of all iterators.
+
+        We perform infer bound using TVM schedule and fill the State with those information. After
+        applying this methods, the State is guaranteed to have complete interator extent
+        information.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.

Review comment:
       ```suggestion
               The state from which we get transform steps
   ```

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`

Review comment:
       ```suggestion
               A `te.schedule` and a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.

Review comment:
       ```suggestion
       and a list of transformation steps used to construct it.
   ```

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.

Review comment:
       ```suggestion
               The state from which we get transform steps
   ```
   Do not overuse `target`, it is redundant.

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.

Review comment:
       ```suggestion
       Each State corresponds to a schedule for its ComputeDAG.
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages
+        self.stage_id_map = {}    # A dict maps operation to stage id
+        self._update_stage_id_map()
+
+    @property
+    def stages(self):
+        """
+        Returns
+        -------
+        stages : List[Stage]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return self.stages_cache
+
+    @property
+    def stage_ops(self):
+        """
+        Returns
+        -------
+        ops: List[Operation]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return [stage.op for stage in self.stages_cache]
+
+    def reorder(self, stage, order):
+        """ Schedule primitive corresponds to te.reorder.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be reordered, can be a Stage order index, Stage operation or stage
+            output tensor.
+        order : List[Iterator]
+            Iterators in the expected order
+        """
+        stage_id = self._resolve_stage_id(stage)
+
+        self.state_object = _ffi_api.StateReorder(self.state_object, stage_id, order)
+        self._clear_cache()
+
+    def split(self, stage, iterator, lengths, inner_to_outer=True):
+        """ Schedule primitive corresponds to te.split.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be split, can be a Stage order index, Stage operation or stage
+            output tensor.
+        iterator : Iterator
+            The iterator to split
+        lengths: List[int]
+            The split factors
+        inner_to_outer: bool = True
+            True to use `factor` to split from inner to outer,
+            False to use `nparts` to split from outer to inner
+
+        Returns
+        -------
+        res_its : List[Iterator]
+            The splitted new Iterators
+        """
+        stage_id = self._resolve_stage_id(stage)
+
+        self.state_object, res = _ffi_api.StateSplit(self.state_object, stage_id, iterator, lengths,
+                                                     inner_to_outer)
+        self._clear_cache()
+        return res
+
+    def fuse(self, stage, iters):
+        """ Schedule primitive corresponds to te.fuse.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be fused, can be a Stage order index, Stage operation or stage

Review comment:
       ```suggestion
               The Stage to be fused, can be a Stage order index, Stage operation or stage
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.

Review comment:
       ```suggestion
           The original ComputeDAG of this State.
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.

Review comment:
       ```suggestion
           The StateObject corresponding to C++ internal State object.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The target search task or workload key.

Review comment:
       ```suggestion
           The search task or workload key.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target).




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       @merrymercy Marisa means make the return type something like `Optional<PrimExpr>`




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       I've tried to modify it with Optional<PrimExpr> or Optional<FloatImm>, but seems that makes this part more complicated ...




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       @tqchen Added a temporary implementation of parallel_for 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.

Review comment:
       I see. It seems to me that we should have a `decode` or `deserialize` function for workload key string. Please resolve this and focus the discussion in the one at L170.




----------------------------------------------------------------
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] kevinthesun commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters

Review comment:
       How do we decide these args? Some of them can be auto-detected from OS but what about ```max_unroll_vec``` and ```max_innermost_split_factor```?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.h
##########
@@ -184,22 +184,36 @@ inline void PrintTitle(const std::string& title, int verbose) {
 }
 
 /*!
- * \brief A simple thread pool.
+ * \brief A simple thread pool to perform parallel for.
  * TODO(merrymercy): Move this to `src/support/parallel_for`
  */
-class ThreadPool {
+class ParallelFor {

Review comment:
       Ok, I get it. However after checking the current code, I found out that actually we have removed all the use of ThreadPool in this minimum system. Didn't realize that before.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.

Review comment:
       Doc updated.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       Yes, we should just remove thread pool




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ApplySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class of transformation steps. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */

Review comment:
       Remove all "target" in this file




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.
+   * \param state The target State.
+   */
+  MeasureInput(SearchTask task, State state);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MeasureInput, ObjectRef, MeasureInputNode);
+};
+
+/*! \brief Store the result of a build. */
+class BuildResultNode : public Object {
+ public:
+  /*! \brief The filename of built binary file. */
+  std::string filename;

Review comment:
       String




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]
+        The target search task or workload key.
+    target : Target
+        The target device of this schedule search.
+    target_host : Target = None
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str]
+        The search policy to be used for schedule search.
+    hardware_params : HardwareParams
+        The hardware parameters of this schedule search.
+    tune_option : TuneOption
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    if tune_option is None:
+        tune_option = TuneOption(n_trials=0)

Review comment:
       How about we setting the default value to 1, since 0&1 are the some in search policy code.
   I'll add more explanation in the fuction doc.




----------------------------------------------------------------
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 pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   > Does not have to change now, but let us change the use of ThreadPool to parallel_for abstraction.
   
   Does that means to just modify ThreadPool to ParallelFor now? Class renamed & added some comments on the member function.
   
   > @jcf94 and @merrymercy thanks for all the hard work! Can I request that we put another unresolved issue? In my opinion the written English parts i.e comments, explanations, etc could still use some improvement with both content and grammar and I would propose in general that we do some at least 1 or 2 rounds of full documentation polish (comments, examples, tests, tutorials, etc) before we officially release a feature (in this case when all of Ansor is landed in master). We tried to do this with Relay but I think we should continue to strive to do a better job with new features like this.
   
   Thanks! That would be of great help since I'm not a native speaker. The documentation does need to be polished.


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):

Review comment:
       You can see the docstring below.
   We use fork to copy arguments between processings with a global variable.
   This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
   




----------------------------------------------------------------
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 pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   @icemelon9 @yzhliu @zhiics @kevinthesun @anijain2305 it would be great if you guys could also help review the core data structure and flow to make sure everything is covered.


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       @tqchen Ok, I understand that(the stride argument has been set to 1 in default in utils.h), and it's fine to further clean these code.
   Just confused about the "does not have to change now" above. :)
   And actually the ThreadPool is never used in current code base...




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       @MarisaKirisame This name is not invented by us. We inherits the builtin class in python and only set the `daemon` property to false. Do you have a better name?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).

Review comment:
       Also propagate the changes to c++ files.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */

Review comment:
       What's the meaning of "target range"?
   Remove all "target" in this file and `loop_state.cc`, `transform_steps.h`, `transform_steps.cc`




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state.
+
+        The states can lose complete bound information after some transform steps
+        (e.g., compute_at).
+        We can call this function to infer and fill all the bound information.
+        This function calls TVM InferBound pass internally to get the bound.
+        The returned state of this function is guaranteed to have complete iterator extent
+        information.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        state : State
+            The State with complete bound information.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return State(_ffi_api.ComputeDAGInferBoundFromState(self, state_obj), self)
+
+    def __hash__(self):
+        # TODO(merrymercy): Implement this more carefully and move this to c++ as a member function

Review comment:
       some c++ functions may need this. It is more convienet for c++ callers.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.

Review comment:
       ```suggestion
           This is used to print transformation steps for debugging.
           Use `apply_steps_from_state` if you want to get a schedule for code generation
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):

Review comment:
       Since this is not used in Ansor minimum system, remove this function for 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] MarisaKirisame commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,470 @@
+# 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.
+
+"""
+Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling.
+"""

Review comment:
       measureinput, buildresult and measureresult are just badname imo.
   what about Program, Kernel, Measurement?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   What's the meaning of the rust error in CI? It appears again after I retriggered the CI.


----------------------------------------------------------------
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] FrozenGene commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/utils.cc
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;

Review comment:
       Maybe we should explain a little bit why we should `REFRESH_EVERY`




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Thank all the reviewers! Most comments from the reviewers are addressed.
   
   We summarize the unresolved comments here. They will be addressed by follow-up PRs. We want to merge this PR first to accelerate the upstream process.
   - The design of `python/tvm/auto_schedule/auto_scheduler.py::auto_schedule`. This is the most important user interface to use the auto-scheduler. We will send an RFC to discuss this.
   - Move `src/auto_schedule/utils.h::ThreadPool` to `src/support/parallle_for.h` (create a new file)
   - Consolidate the usage of `python/tvm/auto_schedule/utils.py::get_const_int`, `python/tvm/auto_schedule/utils.py::get_const_tuple` in the whole TVM code base.
   - Improve the hash of `LoopState` and `TransformStep`. Try to implement it with `SHashReduce`, `SEqualReduce`
   - Improve the log serialization format. Do not use only a hash key as the workload key.
   
   @jroesch @jwfromm @yangjunpro @MarisaKirisame @junrushao1994 @FrozenGene  Please take another look and approve.


----------------------------------------------------------------
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] MarisaKirisame commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,224 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a schedule for its ComputeDAG.

Review comment:
       LoopRepr(sentation)? What matter isnt the mutability, but what the value represent.

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       can we dont use negative? it is not implemented as a daemon pool, so how is it implemented?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):

Review comment:
       Update to more clear API since the file name has been changed to measure_record.py

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):

Review comment:
       Updated to more clear API since the file name has been changed to measure_record.py




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.

Review comment:
       In c++ files, we can add pointers to the detailed doc string here.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.

Review comment:
       `number`, `repeat`, `min_repeat_ms` are very hard to understand.
   Please copy some doc from 
   https://github.com/apache/incubator-tvm/blob/575a3835315a533a19e871ee913f01142befebab/python/tvm/autotvm/measure/measure_methods.py#L151-L183
   
   In c++ files, we can add pointers to the detailed doc string 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system

Review comment:
       ```suggestion
    * - transform_step.cc: How each step interact with TE and TE's schedule primitives
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """

Review comment:
       +1




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.

Review comment:
       ```suggestion
   These strings are efficient for serialization/matching and won't be too long.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The generation function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    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.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the function.
+    """
+    if callable(func):
+        func_name = func.__name__

Review comment:
       Let write a helper function to get name from a given python func.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+

Review comment:
       I am trying to understand this...Could you check if the following understanding is correct?
   
   A. "Workload" is the workload ansor wants to tune. For exmaple, it can be a matmul, whose inputs are n, m, k (matrix dimensions).
   B. The workload key is used to describe the entire workload, i.e. the entire workload can be completely described using registered function and its inputs. For example, using `n`, `m`, `k`, we can reconstruct the matmul function by re-creating the two corresponding tensors, build a ComputeOp and then return its result.
   C. The computational dag can be created using a workload key.
   
   

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The generation function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__

Review comment:
       This can be problematic when codebase grows, because of potential naming conflicts across different python files. IMHO a better solution would be
   1) Use `func.__qualname__` instead
   2) Allow users to provide a customized name, like what is done in `tvm._ffi.register_func`

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_by_func(func):
+    """ Register a workload by generation function.

Review comment:
       I found "generation function" confusing too...Not a native speaker who may provide professional suggestions, but I guess it would be slightly better to say "Register a function who generates a certain workload"




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       @merrymercy Marisa means make the return type something like “Optional<PrimExpr>”




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {

Review comment:
        let us change the namespace to `auto_schedule`, so that the module can be a generic module of tvm.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {

Review comment:
       Yes, in design we may set the length to be None and left for Search Policy to fill.




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.cc
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.cc
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step.
+ */
+
+#include "transform_step.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+/********** Reorder **********/
+ReorderStep::ReorderStep(int stage_id, const Array<PrimExpr>& after_ids) {
+  auto node = make_object<ReorderStepNode>();
+  node->stage_id = stage_id;
+  for (const auto& x : after_ids) {
+    CHECK(x.defined() && x->IsInstance<IntImmNode>());
+  }
+  node->after_ids = after_ids;
+  data_ = std::move(node);
+}
+
+void ReorderStepNode::ApplyToSchedule(Array<te::Stage>* stages,
+                                      StageToAxesMap* stage_to_axes) const {
+  auto stage = (*stages)[stage_id];
+  const Array<IterVar>& axes = stage_to_axes->at(stage);
+  CHECK_EQ(after_ids.size(), axes.size());
+
+  Array<IterVar> new_axes;
+  new_axes.reserve(axes.size());
+  for (auto i : after_ids) {
+    new_axes.push_back(axes[i.as<IntImmNode>()->value]);
+  }
+  stage.reorder(new_axes);
+
+  stage_to_axes->Set(stage, std::move(new_axes));
+  stages->Set(stage_id, std::move(stage));
+}
+
+String ReorderStepNode::PrintAsPythonAPI(Array<te::Stage>* stages,
+                                         StageToAxesMap* stage_to_axes) const {
+  const auto& stage = (*stages)[stage_id];
+  std::stringstream ss;
+
+  ss << "s[" << CleanName(stage->op->name) << "].reorder(";
+  for (size_t i = 0; i < after_ids.size(); ++i) {
+    ss << CleanName((*stage_to_axes)[stage][after_ids[i].as<IntImmNode>()->value]->var->name_hint);
+    if (i != after_ids.size() - 1) {
+      ss << ", ";
+    }
+  }
+  ss << ")\n";
+
+  ApplyToSchedule(stages, stage_to_axes);
+  return ss.str();
+}
+
+/********** Split **********/
+Array<IterVar> ApplySplitToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes,
+                                    int stage_id, int iter_id, const Array<PrimExpr>& lengths,
+                                    bool inner_to_outer) {
+  auto stage = (*stages)[stage_id];
+  const Array<IterVar>& axes = stage_to_axes->at(stage);
+
+  Array<IterVar> outs;
+  if (inner_to_outer) {
+    IterVar outer = axes[iter_id], inner;
+    for (int i = static_cast<int>(lengths.size()) - 1; i >= 0; i--) {
+      IterVar to_split = outer;
+      stage.split(to_split, lengths[i], &outer, &inner);
+      outs.push_back(inner);
+    }
+    outs.push_back(outer);
+  } else {
+    IterVar outer, inner = axes[iter_id];
+    for (size_t i = 0; i < lengths.size(); i++) {
+      IterVar to_split = inner;
+      stage.split_by_nparts(to_split, lengths[i], &outer, &inner);
+      outs.push_back(outer);
+    }
+    outs.push_back(inner);
+  }
+
+  Array<IterVar> new_axes;
+  new_axes.insert(new_axes.end(), axes.begin(), axes.begin() + iter_id);
+  if (inner_to_outer) {
+    for (auto x = outs.rbegin(); x != outs.rend(); ++x) {
+      new_axes.push_back((*x));
+    }
+  } else {
+    for (const auto& x : outs) {
+      new_axes.push_back(x);
+    }
+  }
+  new_axes.insert(new_axes.end(), axes.begin() + iter_id + 1, axes.end());
+
+  stage_to_axes->Set(stage, std::move(new_axes));
+  stages->Set(stage_id, std::move(stage));
+  return outs;
+}
+
+String PrintSplitAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes, int stage_id,
+                             int iter_id, const Array<PrimExpr>& lengths, bool inner_to_outer) {
+  const auto& stage = (*stages)[stage_id];
+  auto to_split = stage_to_axes->at(stage)[iter_id];
+  const auto& func_name = CleanName(stage->op->name);
+  const auto& outs =
+      ApplySplitToSchedule(stages, stage_to_axes, stage_id, iter_id, lengths, inner_to_outer);
+  CHECK_EQ(outs.size(), lengths.size() + 1);
+
+  std::stringstream ss;
+  int size = static_cast<int>(lengths.size());
+  if (inner_to_outer) {
+    for (int i = size - 1; i >= 0; i--) {
+      ss << CleanName(outs[size - i]->var->name_hint) << ", "
+         << CleanName(outs[size - i - 1]->var->name_hint) << " = s[" << func_name << "].split("
+         << CleanName(to_split->var->name_hint) << ", factor=" << lengths[i] << ")\n";
+      to_split = outs[size - i];
+    }
+  } else {
+    for (int i = 0; i < size; i++) {
+      ss << CleanName(outs[i]->var->name_hint) << ", " << CleanName(outs[i + 1]->var->name_hint)
+         << " = s[" << func_name << "].split(" << CleanName(to_split->var->name_hint)
+         << ", nparts=" << lengths[i] << ")\n";
+      to_split = outs[i + 1];
+    }
+  }
+
+  return ss.str();
+}
+
+SplitStep::SplitStep(int stage_id, int iter_id, PrimExpr extent, const Array<PrimExpr>& lengths,
+                     bool inner_to_outer) {
+  auto node = make_object<SplitStepNode>();
+  node->stage_id = stage_id;
+  // Extent can be a unreducible expression in some special cases
+  if (extent->IsInstance<IntImmNode>()) {
+    node->extent = std::move(extent);
+  }
+  node->iter_id = iter_id;
+  node->lengths = lengths;
+  node->inner_to_outer = inner_to_outer;
+  data_ = std::move(node);
+}
+
+Array<IterVar> SplitStepNode::ApplyToSchedule(Array<te::Stage>* stages,
+                                              StageToAxesMap* stage_to_axes) const {
+  return ApplySplitToSchedule(stages, stage_to_axes, stage_id, iter_id, lengths, inner_to_outer);
+}
+
+String SplitStepNode::PrintAsPythonAPI(Array<te::Stage>* stages,
+                                       StageToAxesMap* stage_to_axes) const {
+  return PrintSplitAsPythonAPI(stages, stage_to_axes, stage_id, iter_id, lengths, inner_to_outer);
+}
+
+/********** Fuse **********/
+FuseStep::FuseStep(int stage_id, const Array<PrimExpr>& fused_ids) {

Review comment:
       `Array<Integer>`

##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ReplaySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add hash support in `struct hash<::tvm::ansor::Step>`. (search for this function in this file)
+ * 7. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class for a transformation step. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */
+  int stage_id;
+
+  static constexpr const char* _type_key = "ansor.Step";
+  TVM_DECLARE_BASE_OBJECT_INFO(StepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StepNode.
+ * \sa StepNode
+ */
+class Step : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(Step, ObjectRef, StepNode);
+};
+
+/*! \brief Reorder step that corresponds to te::Stage::reorder */
+class ReorderStepNode : public StepNode {
+ public:
+  /*!
+   * \brief The iterator ids after reorder.
+   * This array should specify the order of all iterators.
+   */
+  Array<PrimExpr> after_ids;

Review comment:
       `Array<Integer>`




----------------------------------------------------------------
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] zhiics commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):

Review comment:
       yeah, nvm, I was thinking if it is necessary to have `_from _state` for these APIs.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/utils.cc
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;

Review comment:
       @tqchen Is there a thread pool in the code base that supports our usage? We want to submit an arbitrary function to a thread pool for parallel execution




----------------------------------------------------------------
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] MarisaKirisame commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1

Review comment:
       verbosity: bool = true

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1

Review comment:
       Option[int]




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/auto_schedule.py
##########
@@ -0,0 +1,194 @@
+# 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.
+
+"""
+User interface for TVM Auto-scheduler.
+
+The basic schedule search process for TVM Auto-scheduler is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("auto_schedule.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search policy
+
+    TODO(jcf94): This is considered to be merged with the new Target:

Review comment:
       ```suggestion
       TODO(jcf94): This is considered to be merged with the new Target specification:
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       I would like to leave it as follow-up PRs.

##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       I would like to leave it to follow-up PRs.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.

Review comment:
       ```suggestion
       the init state of input ComputeDAG.
   ```
   
   You can safely remove all "target" before "ComputeDAG", "compute dag", "compute decleration", "SearchTask", "search task", "task", "function", "State", "Stage" in all files. They are redundant.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});

Review comment:
       Updated this to use all ops as input. Will this cause other bugs? @merrymercy 




----------------------------------------------------------------
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] tqchen edited a comment on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Folks, please another look and https://tvm.apache.org/docs/contribute/code_review.html#approve-and-request-changes-explicitly


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):

Review comment:
       Update the class name to `TuningOptions`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/auto_schedule.py
##########
@@ -0,0 +1,194 @@
+# 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.
+
+"""
+User interface for TVM Auto-scheduler.
+
+The basic schedule search process for TVM Auto-scheduler is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("auto_schedule.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.

Review comment:
       ```suggestion
       """ The parameters of target hardware used to guide the search policy
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */

Review comment:
       What's the meaning of "target range"?
   Remove all "target" in `loop_state.py`, `loop_state.h`, `loop_state.cc`, `transform_steps.h`, `transform_steps.cc`, 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.

Review comment:
       ```suggestion
           The ComputeDAG for the compute declaration.
   ```
   
   Do not use `target` too much because it already has its meaning.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.

Review comment:
       The workload_key here is a string with special format, for example a string of `'["matmul_ansor_test", 128, 128, 128]'`.
   So this function dose do decode first, and then use the decoded result to look up in registry.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.

Review comment:
       ```suggestion
           This is used to print transforms steps for debugging.
           Use `apply_steps_from_state` if you want to get a schedule for code generation
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: tests/python/unittest/test_ansor_search_policy.py
##########
@@ -0,0 +1,86 @@
+# 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.
+
+"""Test search policy"""
+
+import random
+import numpy as np
+import tempfile
+
+import tvm
+from tvm import ansor
+
+from test_ansor_common import matmul_ansor_test, PropagatingThread
+
+def search_common(target="llvm", seed=random.randint(1, 1 << 30), runner='local',
+                  cost_model=None, num_measure_trials=2, params=None,
+                  pre_search_callbacks=None):
+    print("Test %s schedule search with the default search policy" % (target))
+
+    random.seed(seed)
+    N = 128
+    workload_key = ansor.make_workload_key(matmul_ansor_test, (N, N, N))
+    dag = ansor.ComputeDAG(workload_key)
+    target = tvm.target.create(target)
+    task = ansor.SearchTask(dag, workload_key, target)
+
+    with tempfile.NamedTemporaryFile() as fp:
+        log_file = fp.name
+
+        search_policy = ansor.EmptyPolicy()
+        # search_policy = ansor.SketchSearchPolicy(cost_model, params=params, seed=seed)
+        tuning_options = ansor.TuningOptions(num_measure_trials=num_measure_trials, runner=runner,
+                                             verbose=0,
+                                             measure_callbacks=[ansor.LogToFile(log_file)],
+                                             pre_search_callbacks=pre_search_callbacks)
+        sch, args = ansor.auto_schedule(task, target, search_policy=search_policy,
+                                        tuning_options=tuning_options)
+        inp, res = ansor.best_measure_pair_in_file(log_file, workload_key, target)
+
+        print("==== Python Code ====")
+        print(dag.print_python_code_from_state(inp.state))
+
+        try:
+            print("==== Lowered Stmt ====")
+            print(tvm.lower(sch, args, simple_mode=True))
+            mod = tvm.build(sch, args, target)
+
+            ctx = tvm.context(str(target), 0)
+            dtype = dag.tensors[0].dtype
+            a = tvm.nd.array(np.random.uniform(size=(N, N)).astype(dtype), ctx)
+            b = tvm.nd.array(np.random.uniform(size=(N, N)).astype(dtype), ctx)
+            c = tvm.nd.array(np.zeros((N, N), dtype=dtype), ctx)
+            mod(a, b, c)
+            tvm.testing.assert_allclose(c.asnumpy(), np.dot(
+                a.asnumpy(), b.asnumpy()), rtol=1e-5)
+            print("==== Verification passed ====")
+        except Exception:
+            raise Exception("Error encountered with seed: %d" % (seed))
+    print()
+
+
+def test_search_basic():
+    if not tvm.runtime.enabled("llvm"):
+        return

Review comment:
       That's because this unit test requires LLVM backend. This is common in unit tests to have similar checks, and it should be fine because we know that LLVM is always enabled in CI.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Thank all the reviewers! Most comments from the reviewers are addressed.
   
   We summarize the unresolved comments here. They will be addressed by follow-up PRs. We want to merge this PR first to accelerate the upstream process.
   - The design of `python/tvm/auto_schedule/auto_scheduler.py::auto_schedule`. This is the most important user interface to use the auto-scheduler. We will send an RFC to discuss this.
   - Move `src/auto_schedule/utils.h::ThreadPool` to `src/support/parallle_for.h` (create a new file)
   - Consolidate the usage of `python/tvm/auto_schedule/utils.py::get_const_int`, `python/tvm/auto_schedule/utils.py::get_const_tuple` in the whole TVM code base.
   
   @jroesch @jwfromm @yangjunpro @MarisaKirisame @junrushao1994 @FrozenGene  Please take another look and approve.


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/record.py
##########
@@ -0,0 +1,157 @@
+# 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.
+
+""" Serialization and other I/O support for tuning logs (measurement records). """
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=None, skip_lines=None):

Review comment:
       Rewrited `max_lines` to use None in default and `skip_lines` to use 0 in default.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,379 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Str
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    time_cost : Float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename if filename else "", args, error_no,
+            error_msg if error_msg else "", time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[Float]
+        The time costs of execution.
+    error_no : Int
+        The error code.
+    error_msg : Str
+        The error message if there is any error.
+    all_cost : Float
+        The time cost of build and run.
+    timestamp : Float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg if error_msg else "", all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : Int
+            Verbosity level. (0 means silent)
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each build.
+    n_parallel : Int
+        Number of threads used to build in parallel.
+    build_func : Str
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : Int
+        The timeout limit for each run.
+    number : Int
+        Number of measure times.
+    repeat : Int
+        Number of repeat times in each measure.
+    min_repeat_ms : Int
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : Float
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+GLOBAL_BUILD_ARGUMENTS = None
+GLOBAL_RUN_ARGUMENTS = None
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=W0703
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=W0703
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose >= 1:

Review comment:
       Seems the first one is better for dealing with these output information.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.

Review comment:
       ```suggestion
               The state from which we get transform steps
   ```

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`

Review comment:
       ```suggestion
               A `te.schedule` and 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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */

Review comment:
       ```suggestion
     /*! \brief The range of this iterator. */
   ```




----------------------------------------------------------------
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 pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Most of the commemts have been addressed, expect for some controversies.


----------------------------------------------------------------
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 pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   > Thank you for the contribution! Just did a round of review on the c++ side. Code is pretty well written and organized - really enjoy it when reading them through. Comments are mostly nitpicks, and feel free to ignore if they don't make sense :-)
   
   Thanks! It's really kind of you to provides these helpful comments!
   I'll take some time to fix them 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 a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]
+        The target search task or workload key.
+    target : Target
+        The target device of this schedule search.
+    target_host : Target = None
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str]
+        The search policy to be used for schedule search.
+    hardware_params : HardwareParams
+        The hardware parameters of this schedule search.
+    tune_option : TuneOption
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    if tune_option is None:
+        tune_option = TuneOption(n_trials=0)

Review comment:
       n_trials = 0 means we do the search but don't do measurement




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("stages", &stages);
+    v->Visit("transform_steps", &transform_steps);
+    v->Visit("complete", &complete);
+  }
+
+  static constexpr const char* _type_key = "ansor.State";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StateNode, Object);
+
+ private:
+  /*!
+   * \brief The up-to-date ComputeDAG of this state, used for some steps that may change the
+   * stage structure of the ComputeDAG, for exp. CacheReadStep/CacheWriteStep(Will be added later).
+   * The default value is an empty ObjectRef. (means no modification to the original DAG)
+   */
+  ObjectRef current_compute_dag;
+};
+
+/*!
+ * \brief Managed reference to StateNode.
+ * \sa StateNode
+ */
+class State : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param ops `te::Operation`s for a compute declaration.
+   */
+  explicit State(const Array<te::Operation>& ops);
+
+  /*!
+   * \brief Print the state to a human readable string.
+   * \param delete_trivial_loop True for skipping the trivial loops.
+   * (undefined or extent == 1, default set to True)
+   * \return The human readable state structure.
+   */
+  String ToStr(bool delete_trivial_loop = true) const;
+
+  /*!
+   * \brief General do step functions with a runtime dynamic dispatcher.
+   * \param dag The target ComputeDAG.
+   */
+  void DoSteps(const ComputeDAG& dag);
+
+  /* Step APIs for State. */
+
+  /*!
+   * \brief Schedule primitive corresponds to te.reorder.
+   * \param stage_id The index of the target stage.

Review comment:
       remove 'target' from the docstrings for these three functions (reorder, split and fuse). See my comments on the python file

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.

Review comment:
       Improve the docstring by using my comments on the python file.

##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <unordered_map>
+#include <utility>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.

Review comment:
       remove all "target" before "SearchTask", "State" in all files




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <unordered_map>
+#include <utility>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.

Review comment:
       remove all "target" before "SearchTask", "State" in all files




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The target search task or workload key.
+    target : tvm.target.Target
+        The target device of this schedule search.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str] = 'default'
+        The search policy to be used for schedule search.
+    hardware_params : Optional[HardwareParams]
+        The hardware parameters of this schedule search.
+    tuning_options : Optional[TuningOptions]
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`

Review comment:
       ```suggestion
           A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`
   ```
   Remove all "target" before `te.Tensors` in all files




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");

Review comment:
       Done.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during schedule searching. */
+  int verbose;
+  /*! \brief ProgramBuilder which builds the program */
+  ProgramBuilder builder;
+  /*! \brief ProgramRunner which runs the program and measure time costs */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback functions to be called after each measure batch */
+  Array<MeasureCallback> measure_callbacks;
+  /*! \brief SearchCallback functions to be called before schedule search */
+  Array<SearchCallback> pre_search_callbacks;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_measure_trials", &num_measure_trials);
+    v->Visit("early_stopping", &early_stopping);
+    v->Visit("num_measures_per_round", &num_measures_per_round);
+    v->Visit("verbose", &verbose);
+    v->Visit("builder", &builder);
+    v->Visit("runner", &runner);
+    v->Visit("measure_callbacks", &measure_callbacks);
+    v->Visit("pre_search_callbacks", &pre_search_callbacks);
+  }
+
+  static constexpr const char* _type_key = "ansor.TuningOptions";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningOptionsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to TuningOptionsNode.
+ * \sa TuningOptionsNode
+ */
+class TuningOptions : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor
+   * \param num_measure_trials Number of total measurement trials.
+   * \param early_stopping Stops early the tuning if no improvement after n measurements.
+   * \param num_measures_per_round The number of programs to be measured at each search round.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during schedule
+   * search.
+   * \param builder ProgramBuilder which builds the program.
+   * \param runner ProgramRunner which runs the program and measure time costs.
+   * \param measure_callbacks MeasureCallback functions to be called after each measure batch.
+   * \param pre_search_callbacks SearchCallback functions to be called before schedule search.
+   */
+  TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round, int verbose,
+                ProgramBuilder builder, ProgramRunner runner,
+                Array<MeasureCallback> measure_callbacks,
+                Array<SearchCallback> pre_search_callbacks);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TuningOptions, ObjectRef, TuningOptionsNode);
+};
+
+/*!
+ * \brief Auto schedule search for a given compute declaration, by SearchTask.
+ * \param task The target search task.

Review comment:
       ```suggestion
    * \param task The search task.
   ```

##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during schedule searching. */
+  int verbose;
+  /*! \brief ProgramBuilder which builds the program */
+  ProgramBuilder builder;
+  /*! \brief ProgramRunner which runs the program and measure time costs */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback functions to be called after each measure batch */
+  Array<MeasureCallback> measure_callbacks;
+  /*! \brief SearchCallback functions to be called before schedule search */
+  Array<SearchCallback> pre_search_callbacks;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_measure_trials", &num_measure_trials);
+    v->Visit("early_stopping", &early_stopping);
+    v->Visit("num_measures_per_round", &num_measures_per_round);
+    v->Visit("verbose", &verbose);
+    v->Visit("builder", &builder);
+    v->Visit("runner", &runner);
+    v->Visit("measure_callbacks", &measure_callbacks);
+    v->Visit("pre_search_callbacks", &pre_search_callbacks);
+  }
+
+  static constexpr const char* _type_key = "ansor.TuningOptions";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningOptionsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to TuningOptionsNode.
+ * \sa TuningOptionsNode
+ */
+class TuningOptions : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor
+   * \param num_measure_trials Number of total measurement trials.
+   * \param early_stopping Stops early the tuning if no improvement after n measurements.
+   * \param num_measures_per_round The number of programs to be measured at each search round.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during schedule
+   * search.
+   * \param builder ProgramBuilder which builds the program.
+   * \param runner ProgramRunner which runs the program and measure time costs.
+   * \param measure_callbacks MeasureCallback functions to be called after each measure batch.
+   * \param pre_search_callbacks SearchCallback functions to be called before schedule search.
+   */
+  TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round, int verbose,
+                ProgramBuilder builder, ProgramRunner runner,
+                Array<MeasureCallback> measure_callbacks,
+                Array<SearchCallback> pre_search_callbacks);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TuningOptions, ObjectRef, TuningOptionsNode);
+};
+
+/*!
+ * \brief Auto schedule search for a given compute declaration, by SearchTask.
+ * \param task The target search task.
+ * \param search_policy The search policy to be used for schedule search.
+ * \param tuning_options Tuning and measurement options.
+ * \return A `te::Schedule` and the target `te::Tensor` to be used in `tvm.lower` or `tvm.build`.

Review comment:
       ```suggestion
    * \return A `te::Schedule` and 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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.

Review comment:
       ```suggestion
           The workload key for the compute declaration.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target)




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]
+        The target search task or workload key.
+    target : Target
+        The target device of this schedule search.
+    target_host : Target = None
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str]
+        The search policy to be used for schedule search.
+    hardware_params : HardwareParams
+        The hardware parameters of this schedule search.
+    tune_option : TuneOption
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    if tune_option is None:
+        tune_option = TuneOption(n_trials=0)
+
+    if isinstance(workload, str):
+        sch, tensors = _ffi_api.AutoScheduleByWorkloadKey(
+            workload, target, target_host, search_policy, hardware_params, tune_option)
+        return sch, tensors
+    if isinstance(workload, SearchTask):
+        sch, tensors = _ffi_api.AutoScheduleBySearchTask(workload, search_policy, tune_option)
+        return sch, tensors

Review comment:
       Merge these to single entrance.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,103 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import tvm._ffi
+from tvm.runtime import Object
+from .loop_state import State, StateObject
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    tensors : List[Tensor]
+        `Tensor`s for a compute declaration.
+    """
+    def __init__(self, tensors):
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, tensors)
+
+    def get_init_state(self):

Review comment:
       Got it. Let's keep the current one then.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]
+        The target search task or workload key.
+    target : Target
+        The target device of this schedule search.
+    target_host : Target = None
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str]
+        The search policy to be used for schedule search.
+    hardware_params : HardwareParams
+        The hardware parameters of this schedule search.
+    tune_option : TuneOption
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    if tune_option is None:
+        tune_option = TuneOption(n_trials=0)

Review comment:
       n_trials = 0 means we do the search with the cost model but don't do measurements.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.

Review comment:
       In c++ files, we can add pointers to the detailed doc string 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""

Review comment:
       Also, propagate the changes to c++ files.

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.

Review comment:
       Remove all "target" before "StateObject", "State" and "ComputeDAG" in this file.

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages
+        self.stage_id_map = {}    # A dict maps operation to stage id
+        self._update_stage_id_map()
+
+    @property
+    def stages(self):
+        """
+        Returns
+        -------
+        stages : List[Stage]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return self.stages_cache
+
+    @property
+    def stage_ops(self):
+        """
+        Returns
+        -------
+        ops: List[Operation]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return [stage.op for stage in self.stages_cache]
+
+    def reorder(self, stage, order):
+        """ Schedule primitive corresponds to te.reorder.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be reordered, can be a Stage order index, Stage operation or stage

Review comment:
       Remove all "target" before "stage". It is redundant.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.

Review comment:
       Also, add pointers to the docstring in c++ files.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*!
+ * \brief The computation information and hardware parameters for a specific schedule search task.
+ */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */

Review comment:
       replace all "target compute declaration" with "input compute declaration" in all files




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]
+        The target search task or workload key.
+    target : Target
+        The target device of this schedule search.
+    target_host : Target = None
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str]
+        The search policy to be used for schedule search.
+    hardware_params : HardwareParams
+        The hardware parameters of this schedule search.
+    tune_option : TuneOption
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    if tune_option is None:
+        tune_option = TuneOption(n_trials=0)

Review comment:
       Ah I see...this behavior seems not trivial. For example, users may ask what would be the stopping criteria if we don't do any measurement.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):

Review comment:
       Move this function to ComputeDAG's __hash__ methods.

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):

Review comment:
       Moved this function to ComputeDAG's __hash__ methods.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function

Review comment:
       We can doc this to make it clear




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""

Review comment:
       We can rename it to `record.py`

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.

Review comment:
       ```suggestion
       # We use fork and a global variable to copy arguments between processings.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The target function that returns the compute declaration Tensors.

Review comment:
       ```suggestion
           The function that returns the input and output tensors of the compute declaration.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The target function that returns the compute declaration Tensors.
+        Can be the a function or the function name.
+    args : Args
+        The args of the target function.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the target function.
+    """
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    if not func_name in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. "  % func,
+                         "Please register it with @ansor.register_workload")
+
+    args = serialize_args(args)
+
+    return json.dumps((func_name,) + args)
+
+
+def decode_workload_key_to_func_args(workload_key):
+    """ Decode a workload key to the registerd function name and its corresponding args.
+
+    Parameters
+    ----------
+    workload_key : str
+        The target workload key.

Review comment:
       ```suggestion
           The workload key.
   ```

##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+ * a list of all related operations in topo order as well as a set of analyses over each operation
+ * stage (e.g. the total float operation count, consumer/producer relations of each operation
+ * stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+ * help the search policy to do some specific decisions during schedule search process.
+ *
+ * ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+ * (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+ * information get from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+  // TODO(merrymercy): Add more analyses later.
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the
+   * equivalent `tvm::schedule`.
+   * \param transform_steps Transform steps of the target state.
+   * \param stages A pointer to a `te::Stage` Array, default to be nullptr.
+   * Pass a valid pointer if these information needs to be used outside this function.
+   * \param stage_to_axes A pointer to a StageToAxesMap, default to be nullptr.
+   * Pass a valid pointer if these information needs to be used outside this function.
+   * \return The return values can be used as arguments to `tvm.build` or `tvm.lower`.
+   */
+  std::pair<te::Schedule, Array<te::Tensor> > ApplySteps(
+      const Array<Step>& transform_steps, Array<te::Stage>* stages = nullptr,
+      StageToAxesMap* stage_to_axes = nullptr) const;
+
+  /*!
+   * \brief Print transform steps as equivalent python schedule API.
+   * \param transform_steps Transform steps of the target state.

Review comment:
       You can safely delete all "target" in your docstring. It is redundant.

##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+ * a list of all related operations in topo order as well as a set of analyses over each operation
+ * stage (e.g. the total float operation count, consumer/producer relations of each operation
+ * stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+ * help the search policy to do some specific decisions during schedule search process.
+ *
+ * ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+ * (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+ * information get from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+  // TODO(merrymercy): Add more analyses later.
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the
+   * equivalent `tvm::schedule`.
+   * \param transform_steps Transform steps of the target state.
+   * \param stages A pointer to a `te::Stage` Array, default to be nullptr.
+   * Pass a valid pointer if these information needs to be used outside this function.
+   * \param stage_to_axes A pointer to a StageToAxesMap, default to be nullptr.
+   * Pass a valid pointer if these information needs to be used outside this function.
+   * \return The return values can be used as arguments to `tvm.build` or `tvm.lower`.
+   */
+  std::pair<te::Schedule, Array<te::Tensor> > ApplySteps(
+      const Array<Step>& transform_steps, Array<te::Stage>* stages = nullptr,
+      StageToAxesMap* stage_to_axes = nullptr) const;
+
+  /*!
+   * \brief Print transform steps as equivalent python schedule API.
+   * \param transform_steps Transform steps of the target state.
+   * \return Python schedule code.
+   */
+  String PrintStepsAsPython(const Array<Step>& transform_steps) const;
+
+  /*!
+   * \brief Fill the correct bound information for a given state by calling ir_pass::InferBound.
+   * \param state The target state.

Review comment:
       ```suggestion
      * \param state The input state.
   ```

##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <unordered_map>
+#include <utility>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.

Review comment:
       remove target

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("stages", &stages);
+    v->Visit("transform_steps", &transform_steps);
+    v->Visit("complete", &complete);
+  }
+
+  static constexpr const char* _type_key = "ansor.State";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StateNode, Object);
+
+ private:
+  /*!
+   * \brief The up-to-date ComputeDAG of this state, used for some steps that may change the
+   * stage structure of the ComputeDAG, for exp. CacheReadStep/CacheWriteStep(Will be added later).
+   * The default value is an empty ObjectRef. (means no modification to the original DAG)
+   */
+  ObjectRef current_compute_dag;
+};
+
+/*!
+ * \brief Managed reference to StateNode.
+ * \sa StateNode
+ */
+class State : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param ops `te::Operation`s for a compute declaration.
+   */
+  explicit State(const Array<te::Operation>& ops);
+
+  /*!
+   * \brief Print the state to a human readable string.
+   * \param delete_trivial_loop True for skipping the trivial loops.
+   * (undefined or extent == 1, default set to True)
+   * \return The human readable state structure.
+   */
+  String ToStr(bool delete_trivial_loop = true) const;
+
+  /*!
+   * \brief General do step functions with a runtime dynamic dispatcher.
+   * \param dag The target ComputeDAG.

Review comment:
       ```suggestion
      * \param dag The input ComputeDAG.
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    if not GLOBAL_BUILD_ARGUMENTS:
+        raise ValueError("GLOBAL_BUILD_ARGUMENTS not found")
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=broad-except
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=broad-except
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose == 1:
+            if error_no == MeasureErrorNo.NO_ERROR:
+                print(".", end="")
+            else:
+                print(".E", end="")  # Build error
+        return filename, args, error_no, error_msg, time.time() - tic
+
+    res = call_func_with_timeout(timeout, timed_func)
+    if isinstance(res, TimeoutError):
+        if verbose == 1:
+            print(".T", end="")  # Build timeout
+        res = None, [], MeasureErrorNo.BUILD_TIMEOUT, None, timeout
+
+    return res
+
+
+@tvm._ffi.register_func("ansor.local_builder.build")
+def local_builder_build(inputs, timeout, n_parallel, build_func, verbose):
+    """ Local builder build function. """
+    # We use fork to copy arguments from a global variable.

Review comment:
       ```suggestion
       # We use fork and a global variable to copy arguments between processings.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.

Review comment:
       ```suggestion
           The function that returns the compute declaration Tensors.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.

Review comment:
       We do have version number in the log format

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The target function that returns the compute declaration Tensors.
+        Can be the a function or the function name.
+    args : Args
+        The args of the target function.

Review comment:
       ```suggestion
           The args of the function.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_by_func(func):
+    """ Register a workload by generation function.

Review comment:
       This is similar to the mechanism in autotvm.
   When the return values of a function are hard to serialize.
   We serialize the return value by the name of the function and the arguments.
   We assume with the same name and the same set of arguments, the function always returns the same thing.
   We can call this function a "generator"

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.

Review comment:
       ```suggestion
           The MeasureInputs to be written.
   ```

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.
+    results: List[MeasureResults]
+        The target MeasureResults to be written.
+    """
+    _ffi_api.AppendMeasureRecordsToFile(filename, inputs, results)
+
+def best_measure_pair_in_file(filename, workload_key=None, target=None):
+    """ Return the best measurement pair form a log file. This may return none results if
+    there is no legal measure pair with the specified workload_key/target found from the log file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+    workload_key : Optional[str] = None
+        The workload key of the target compute declaration.

Review comment:
       ```suggestion
           The workload key of the compute declaration.
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.

Review comment:
       ```suggestion
           The SearchTask.
   ```

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.
+    results: List[MeasureResults]
+        The target MeasureResults to be written.

Review comment:
       ```suggestion
           The MeasureResults to be written.
   ```

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.

Review comment:
       ```suggestion
           The input function.
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.

Review comment:
       `number`, `repeat`, `min_repeat_ms` are very hard to understand.
   Please copy some doc from 
   https://github.com/apache/incubator-tvm/blob/575a3835315a533a19e871ee913f01142befebab/python/tvm/autotvm/measure/measure_methods.py#L151-L183

##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+ * a list of all related operations in topo order as well as a set of analyses over each operation
+ * stage (e.g. the total float operation count, consumer/producer relations of each operation
+ * stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+ * help the search policy to do some specific decisions during schedule search process.
+ *
+ * ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+ * (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+ * information get from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+  // TODO(merrymercy): Add more analyses later.
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the

Review comment:
       Improve the docstring by using my comments on the python file.

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       Is there anything unclear?
   As documented, this allows us to launch new processings inside the worker function of a `multiprocessing.Pool`

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The target function that returns the compute declaration Tensors.
+        Can be the a function or the function name.
+    args : Args
+        The args of the target function.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the target function.
+    """
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    if not func_name in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. "  % func,
+                         "Please register it with @ansor.register_workload")
+
+    args = serialize_args(args)
+
+    return json.dumps((func_name,) + args)
+
+
+def decode_workload_key_to_func_args(workload_key):
+    """ Decode a workload key to the registerd function name and its corresponding args.
+
+    Parameters
+    ----------
+    workload_key : str
+        The target workload key.
+
+    Returns
+    -------
+    name : str
+        The function name of this workload key.
+    args : List[Tensor]
+        The args of the generation function.
+    """
+    workload = json.loads(workload_key)
+    if not workload[0] in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. " % workload[0] +
+                         "Please register it with @ansor.register_workload")
+    return workload[0], deserialize_args(workload[1:])
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Get the input/output tensors from the workload key.
+
+    This method is usually used to create a ComputeDAG by workload key.
+
+    Parameters
+    ----------
+    workload_key : str
+        The target workload key.

Review comment:
       ```suggestion
           The workload key.
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Union[Function, str]
+        The target function that returns the compute declaration Tensors.
+        Can be the a function or the function name.
+    args : Args
+        The args of the target function.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the target function.

Review comment:
       ```suggestion
           The workload key of the task function.
   ```

##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during schedule searching. */
+  int verbose;
+  /*! \brief ProgramBuilder which builds the program */
+  ProgramBuilder builder;
+  /*! \brief ProgramRunner which runs the program and measure time costs */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback functions to be called after each measure batch */
+  Array<MeasureCallback> measure_callbacks;
+  /*! \brief SearchCallback functions to be called before schedule search */
+  Array<SearchCallback> pre_search_callbacks;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_measure_trials", &num_measure_trials);
+    v->Visit("early_stopping", &early_stopping);
+    v->Visit("num_measures_per_round", &num_measures_per_round);
+    v->Visit("verbose", &verbose);
+    v->Visit("builder", &builder);
+    v->Visit("runner", &runner);
+    v->Visit("measure_callbacks", &measure_callbacks);
+    v->Visit("pre_search_callbacks", &pre_search_callbacks);
+  }
+
+  static constexpr const char* _type_key = "ansor.TuningOptions";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningOptionsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to TuningOptionsNode.
+ * \sa TuningOptionsNode
+ */
+class TuningOptions : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor
+   * \param num_measure_trials Number of total measurement trials.
+   * \param early_stopping Stops early the tuning if no improvement after n measurements.
+   * \param num_measures_per_round The number of programs to be measured at each search round.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during schedule
+   * search.
+   * \param builder ProgramBuilder which builds the program.
+   * \param runner ProgramRunner which runs the program and measure time costs.
+   * \param measure_callbacks MeasureCallback functions to be called after each measure batch.
+   * \param pre_search_callbacks SearchCallback functions to be called before schedule search.
+   */
+  TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round, int verbose,
+                ProgramBuilder builder, ProgramRunner runner,
+                Array<MeasureCallback> measure_callbacks,
+                Array<SearchCallback> pre_search_callbacks);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TuningOptions, ObjectRef, TuningOptionsNode);
+};
+
+/*!
+ * \brief Auto schedule search for a given compute declaration, by SearchTask.
+ * \param task The target search task.

Review comment:
       ```suggestion
    * \param task The search task.
   ```

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.

Review comment:
       Improve the docstring by using my comments on the python file.

##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during schedule searching. */
+  int verbose;
+  /*! \brief ProgramBuilder which builds the program */
+  ProgramBuilder builder;
+  /*! \brief ProgramRunner which runs the program and measure time costs */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback functions to be called after each measure batch */
+  Array<MeasureCallback> measure_callbacks;
+  /*! \brief SearchCallback functions to be called before schedule search */
+  Array<SearchCallback> pre_search_callbacks;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_measure_trials", &num_measure_trials);
+    v->Visit("early_stopping", &early_stopping);
+    v->Visit("num_measures_per_round", &num_measures_per_round);
+    v->Visit("verbose", &verbose);
+    v->Visit("builder", &builder);
+    v->Visit("runner", &runner);
+    v->Visit("measure_callbacks", &measure_callbacks);
+    v->Visit("pre_search_callbacks", &pre_search_callbacks);
+  }
+
+  static constexpr const char* _type_key = "ansor.TuningOptions";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningOptionsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to TuningOptionsNode.
+ * \sa TuningOptionsNode
+ */
+class TuningOptions : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor
+   * \param num_measure_trials Number of total measurement trials.
+   * \param early_stopping Stops early the tuning if no improvement after n measurements.
+   * \param num_measures_per_round The number of programs to be measured at each search round.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during schedule
+   * search.
+   * \param builder ProgramBuilder which builds the program.
+   * \param runner ProgramRunner which runs the program and measure time costs.
+   * \param measure_callbacks MeasureCallback functions to be called after each measure batch.
+   * \param pre_search_callbacks SearchCallback functions to be called before schedule search.
+   */
+  TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round, int verbose,
+                ProgramBuilder builder, ProgramRunner runner,
+                Array<MeasureCallback> measure_callbacks,
+                Array<SearchCallback> pre_search_callbacks);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TuningOptions, ObjectRef, TuningOptionsNode);
+};
+
+/*!
+ * \brief Auto schedule search for a given compute declaration, by SearchTask.
+ * \param task The target search task.
+ * \param search_policy The search policy to be used for schedule search.
+ * \param tuning_options Tuning and measurement options.
+ * \return A `te::Schedule` and the target `te::Tensor` to be used in `tvm.lower` or `tvm.build`.

Review comment:
       ```suggestion
    * \return A `te::Schedule` and a list of `te::Tensor` to be used in `tvm.lower` or `tvm.build`.
   ```

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.

Review comment:
       ```suggestion
      * \param range The range of this iterator.
   ```

##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.

Review comment:
       Improve the docstring by using my comments on the python file

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */

Review comment:
       ```suggestion
     /*! \brief The range of this iterator. */
   ```

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.

Review comment:
       ```suggestion
    * Each State corresponds to a specific schedule for its ComputeDAG.
   ```

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*!
+ * \brief The computation information and hardware parameters for a specific schedule search task.
+ */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */
+  ComputeDAG compute_dag;
+  /*! \brief The workload key for target compute declaration. */
+  String workload_key;

Review comment:
       remove target 

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("stages", &stages);
+    v->Visit("transform_steps", &transform_steps);
+    v->Visit("complete", &complete);
+  }
+
+  static constexpr const char* _type_key = "ansor.State";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StateNode, Object);
+
+ private:
+  /*!
+   * \brief The up-to-date ComputeDAG of this state, used for some steps that may change the
+   * stage structure of the ComputeDAG, for exp. CacheReadStep/CacheWriteStep(Will be added later).
+   * The default value is an empty ObjectRef. (means no modification to the original DAG)
+   */
+  ObjectRef current_compute_dag;
+};
+
+/*!
+ * \brief Managed reference to StateNode.
+ * \sa StateNode
+ */
+class State : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param ops `te::Operation`s for a compute declaration.
+   */
+  explicit State(const Array<te::Operation>& ops);
+
+  /*!
+   * \brief Print the state to a human readable string.
+   * \param delete_trivial_loop True for skipping the trivial loops.
+   * (undefined or extent == 1, default set to True)
+   * \return The human readable state structure.
+   */
+  String ToStr(bool delete_trivial_loop = true) const;
+
+  /*!
+   * \brief General do step functions with a runtime dynamic dispatcher.
+   * \param dag The target ComputeDAG.
+   */
+  void DoSteps(const ComputeDAG& dag);
+
+  /* Step APIs for State. */
+
+  /*!
+   * \brief Schedule primitive corresponds to te.reorder.
+   * \param stage_id The index of the target stage.

Review comment:
       remove 'target' from the docstrings for these three functions (reorder, split and fuse). See my comments on the python file

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("stages", &stages);
+    v->Visit("transform_steps", &transform_steps);
+    v->Visit("complete", &complete);
+  }
+
+  static constexpr const char* _type_key = "ansor.State";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StateNode, Object);
+
+ private:
+  /*!
+   * \brief The up-to-date ComputeDAG of this state, used for some steps that may change the
+   * stage structure of the ComputeDAG, for exp. CacheReadStep/CacheWriteStep(Will be added later).
+   * The default value is an empty ObjectRef. (means no modification to the original DAG)
+   */
+  ObjectRef current_compute_dag;
+};
+
+/*!
+ * \brief Managed reference to StateNode.
+ * \sa StateNode
+ */
+class State : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param ops `te::Operation`s for a compute declaration.
+   */
+  explicit State(const Array<te::Operation>& ops);
+
+  /*!
+   * \brief Print the state to a human readable string.
+   * \param delete_trivial_loop True for skipping the trivial loops.
+   * (undefined or extent == 1, default set to True)
+   * \return The human readable state structure.
+   */
+  String ToStr(bool delete_trivial_loop = true) const;
+
+  /*!
+   * \brief General do step functions with a runtime dynamic dispatcher.
+   * \param dag The target ComputeDAG.
+   */
+  void DoSteps(const ComputeDAG& dag);
+
+  /* Step APIs for State. */
+
+  /*!
+   * \brief Schedule primitive corresponds to te.reorder.
+   * \param stage_id The index of the target stage.
+   * \param order The target iterator order.
+   */
+  void reorder(int stage_id, const Array<Iterator>& order);
+  /*!
+   * \brief Schedule primitive corresponds to te.split.
+   * \param stage_id The index of the target stage.
+   * \param it The target iterator.
+   * \param lengths The target split factors. Can be None to be filled by search policy.
+   * \param inner_to_outer True for split from inner to outer & False for outer to inner.
+   * \return The iterator results after split.
+   */
+  Array<Iterator> split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                        bool inner_to_outer = true);
+  /*!
+   * \brief Schedule primitive corresponds to te.fuse.
+   * \param stage_id The index of the target stage.
+   * \param iters The target iterators to be fused.
+   * \return The iterator result after fuse.
+   */
+  Iterator fuse(int stage_id, const Array<Iterator>& iters);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(State, ObjectRef, StateNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StateNode);
+
+ private:
+  /* Do transform steps
+   * Note: The following functions only change loop state but do not change transform_history.
+   * We separate these functions out, so you can call them for replay easily given history steps */
+
+  /*!
+   * \brief Apply reorder step to current state.
+   * \param step A ReorderStep.
+   */
+  void DoReorderStep(const ReorderStep& step);
+  /*!
+   * \brief Apply split step to current state.
+   * \param step A SplitStep.
+   * \return The iterator results after split.
+   */
+  Array<Iterator> DoSplitStep(const SplitStep& step);
+  /*!
+   * \brief Apply fuse step to current state.
+   * \param step A FuseStep.
+   * \return The iterator result after fuse.
+   */
+  Iterator DoFuseStep(const FuseStep& step);
+
+  /*!
+   * \brief Common function for DoSplitStep and DoFollowSplitStep(Will be added later).
+   * \param stage_id The index of the target stage.
+   * \param iter_id The index of the target iterator.
+   * \param lengths The target split factors.

Review comment:
       remove target

##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <unordered_map>
+#include <utility>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.
+   * \param state The target State.
+   */
+  MeasureInput(SearchTask task, State state);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MeasureInput, ObjectRef, MeasureInputNode);
+};
+
+/*! \brief Store the result of a build. */
+class BuildResultNode : public Object {
+ public:
+  /*! \brief The filename of built binary file. */
+  String filename;
+  /*! \brief The arguments. */
+  Array<te::Tensor> args;
+  /*! \brief The error code. (0 means no error, see MeasureErrorNO) */
+  int error_no;
+  /*! \brief The error message if there is any error. */
+  String error_msg;
+  /*! \brief The time cost of build. */
+  double time_cost;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("filename", &filename);
+    v->Visit("args", &args);
+    v->Visit("error_no", &error_no);
+    v->Visit("error_msg", &error_msg);
+    v->Visit("time_cost", &time_cost);
+  }
+
+  static constexpr const char* _type_key = "ansor.BuildResult";
+  TVM_DECLARE_FINAL_OBJECT_INFO(BuildResultNode, Object);
+};
+
+/*!
+ * \brief Managed reference to BuildResultNode.
+ * \sa BuildResultNode
+ */
+class BuildResult : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param filename The filename of built binary file.
+   * \param args The arguments.
+   * \param error_no The error code.
+   * \param error_msg The error message if there is any error.
+   * \param time_cost The time cost of build.
+   */
+  BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+              double time_cost);
+  TVM_DEFINE_OBJECT_REF_METHODS(BuildResult, ObjectRef, BuildResultNode);
+};
+
+/*! \brief Store the results of a measurement. */
+class MeasureResultNode : public Object {
+ public:
+  /*! \brief The time costs of execution. */
+  Array<PrimExpr> costs;
+  /*! \brief The error code. (0 means no error, see MeasureErrorNO) */
+  int error_no;
+  /*! \brief The error message if there is any error. */
+  String error_msg;
+  /*! \brief The time cost of build and run. */
+  double all_cost;
+  /*! \brief The time stamps of this measurement. */
+  double timestamp;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("costs", &costs);
+    v->Visit("error_no", &error_no);
+    v->Visit("error_msg", &error_msg);
+    v->Visit("all_cost", &all_cost);
+    v->Visit("timestamp", &timestamp);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureResult copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureResult";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureResultNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureResultNode.
+ * \sa MeasureResultNode
+ */
+class MeasureResult : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param costs The time costs of execution.
+   * \param error_no The error code.
+   * \param error_msg The error message if there is any error.
+   * \param all_cost The time cost of build and run.
+   * \param timestamp The time stamps of this measurement.
+   */
+  MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                double timestamp);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MeasureResult, ObjectRef, MeasureResultNode);
+};
+
+/*! \brief Bass class of measurement callbacks */
+class MeasureCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Callback function that will be called on measurement input/result pairs
+   * after measurement.
+   * \param policy The current search policy.
+   * \param inputs An Array of MeasureInput.
+   * \param results An Array of MeasureResult.
+   */
+  virtual void Callback(const SearchPolicy& policy, const Array<MeasureInput>& inputs,
+                        const Array<MeasureResult>& results) = 0;
+  static constexpr const char* _type_key = "ansor.MeasureCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(MeasureCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureCallbackNode.
+ * \sa MeasureCallbackNode
+ */
+class MeasureCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(MeasureCallback, ObjectRef, MeasureCallbackNode);
+};
+
+// The base class of ProgramBuilders and ProgramRunners.
+
+/*! \brief ProgramBuilder that builds the programs */
+class ProgramBuilderNode : public Object {
+ public:
+  /*! \brief The number of tasks to run in parallel */
+  int n_parallel;
+  /*! \brief Timeout of a build */
+  int timeout;
+
+  /*!
+   * \brief Build programs and return results.
+   * \param inputs An Array of MeasureInput.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during program building.
+   * \return An Array of MeasureResult.
+   */
+  virtual Array<BuildResult> Build(const Array<MeasureInput>& inputs, int verbose) = 0;
+
+  static constexpr const char* _type_key = "ansor.ProgramBuilder";
+  TVM_DECLARE_BASE_OBJECT_INFO(ProgramBuilderNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ProgramBuilderNode.
+ * \sa ProgramBuilderNode
+ */
+class ProgramBuilder : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(ProgramBuilder, ObjectRef, ProgramBuilderNode);
+};
+
+/*! \brief ProgramRunner that runs the built programs and measure the time cost. */
+class ProgramRunnerNode : public Object {
+ public:
+  /*! \brief Timeout of a run. */
+  int timeout;
+
+  /*!
+   * \brief Run measurement and return results.
+   * \param inputs An Array of MeasureInput.
+   * \param build_results An Array of BuildResult.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during program running.
+   * \return An Array of MeasureResult.
+   */
+  virtual Array<MeasureResult> Run(const Array<MeasureInput>& inputs,
+                                   const Array<BuildResult>& build_results, int verbose) = 0;
+
+  static constexpr const char* _type_key = "ansor.ProgramRunner";
+  TVM_DECLARE_BASE_OBJECT_INFO(ProgramRunnerNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ProgramRunnerNode.
+ * \sa ProgramRunnerNode
+ */
+class ProgramRunner : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(ProgramRunner, ObjectRef, ProgramRunnerNode);
+};
+
+// Implementation of various builders and runners
+
+/*! \brief LocalBuilder use local CPU cores to build programs in parallel */
+class LocalBuilderNode : public ProgramBuilderNode {
+ public:
+  /*! \brief Build function. */
+  String build_func;
+
+  Array<BuildResult> Build(const Array<MeasureInput>& inputs, int verbose) final;
+
+  static constexpr const char* _type_key = "ansor.LocalBuilder";
+  TVM_DECLARE_FINAL_OBJECT_INFO(LocalBuilderNode, ProgramBuilderNode);
+};
+
+/*!
+ * \brief Managed reference to LocalBuilderNode.
+ * \sa LocalBuilderNode
+ */
+class LocalBuilder : public ProgramBuilder {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param timeout The timeout limit for each build.
+   * \param n_parallel Number of threads used to build in parallel.
+   * \param build_func The name of registered build function.
+   */
+  LocalBuilder(int timeout, int n_parallel, const String& build_func);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(LocalBuilder, ProgramBuilder, LocalBuilderNode);
+};
+
+/*! \brief LocalRunner that uses local CPU/GPU to measures the time cost of programs */
+class LocalRunnerNode : public ProgramRunnerNode {
+ public:
+  /*! \brief Number of measure times. */
+  int number;
+  /*! \brief Number of repeat times in each measure. */
+  int repeat;
+  /*! \brief The minimum duration of one repeat in milliseconds. */
+  int min_repeat_ms;
+  /*! \brief The cool down interval between two measurements. */
+  double cooldown_interval;
+
+  Array<MeasureResult> Run(const Array<MeasureInput>& inputs,
+                           const Array<BuildResult>& build_results, int verbose) final;
+
+  static constexpr const char* _type_key = "ansor.LocalRunner";
+  TVM_DECLARE_FINAL_OBJECT_INFO(LocalRunnerNode, ProgramRunnerNode);
+};
+
+/*!
+ * \brief Managed reference to LocalRunnerNode.
+ * \sa LocalRunnerNode
+ */
+class LocalRunner : public ProgramRunner {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param timeout The timeout limit for each run.
+   * \param number Number of measure times.
+   * \param repeat Number of repeat times in each measure.
+   * \param min_repeat_ms The minimum duration of one repeat in milliseconds.
+   * \param cooldown_interval The cool down interval between two measurements.
+   */
+  LocalRunner(int timeout, int number, int repeat, int min_repeat_ms, double cooldown_interval);
+
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(LocalRunner, ProgramRunner, LocalRunnerNode);
+};
+
+/*!
+ * \brief Measurer that measures the time costs of tvm programs
+ * This class combines ProgramBuilder and ProgramRunner, and provides a simpler API */
+class ProgramMeasurerNode : public Object {
+ public:
+  /*! \brief Measured programs counter. */
+  int ct;
+  /*! \brief Continuous error counter. */
+  int error_ct;
+  /*! \brief Workload key to best flops map. */
+  std::unordered_map<String, double> best_flops;
+  /*! \brief Workload key to best state map. */
+  std::unordered_map<String, State> best_state;
+  /*! \brief Workload key to best state's count index map. */
+  std::unordered_map<String, int> best_ct;
+  /*! \brief The ProgramBuilder to build each program. */
+  ProgramBuilder builder;
+  /*! \brief The ProgramRunner to measure each program. */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback to be called after each measure batch. */
+  Array<MeasureCallback> callbacks;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during program measuring. */
+  int verbose;
+  /*! \brief The number of max continuous error. */
+  int max_continous_error;
+
+  /*! \brief Reset book keeping variables */
+  void Reset();
+
+  /*!
+   * \brief Do measurement.
+   * \param task The current SearchTask.
+   * \param policy The current SearchPolicy.
+   * \param inputs The target MeasureInputs.
+   * \param results A pointer to a MeasureResult Array, this is used as output.
+   * \param batch_size Number of programs to be measured in one batch.
+   */
+  void Measure(const SearchTask& task, const SearchPolicy& policy,
+               const Array<MeasureInput>& inputs, Array<MeasureResult>* results,
+               int batch_size = -1);
+  /*!
+   * \brief Do measurement silently.
+   * This API will not print the measure results to screen.
+   * \param task The current SearchTask.
+   * \param inputs The target MeasureInputs.

Review comment:
       remove target 

##########
File path: src/ansor/serialization.h
##########
@@ -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.
+ */
+
+/*!
+ * \file ansor/serialization.h
+ * \brief Json serialization format for dumping and loading tuning records.
+ */
+
+#ifndef TVM_ANSOR_SERIALIZATION_H_
+#define TVM_ANSOR_SERIALIZATION_H_
+
+#include <fstream>
+#include <string>
+#include <utility>
+
+#include "measure.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Callback for logging the input and results of measurements to file */

Review comment:
       Some callbacks are stateful. We need a class to store the stateful varaibles.

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*!
+ * \brief The computation information and hardware parameters for a specific schedule search task.
+ */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */
+  ComputeDAG compute_dag;
+  /*! \brief The workload key for target compute declaration. */
+  String workload_key;
+  /*! \brief The target device of this search task. */
+  Target target;
+  /*! \brief The target host device of this search task. */
+  Target target_host;
+  /*! \brief Hardware parameters used in this search task. */
+  HardwareParams hardware_params;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("compute_dag", &compute_dag);
+    v->Visit("workload_key", &workload_key);
+    v->Visit("target", &target);
+    v->Visit("target_host", &target_host);
+    v->Visit("hardware_params", &hardware_params);
+  }
+
+  static constexpr const char* _type_key = "ansor.SearchTask";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SearchTaskNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchTaskNode.
+ * \sa SearchTaskNode
+ */
+class SearchTask : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param compute_dag The ComputeDAG for target compute declaration.
+   * \param workload_key The workload key for target compute declaration.

Review comment:
       Replace target with input

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):

Review comment:
       Some callbacks are stateful. We need a class to store the stateful varaibles.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {
+  auto node = make_object<MeasureInputNode>();
+  node->task = task;
+  node->state = state;
+  return MeasureInput(node);
+}
+
+BuildResult::BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+                         double time_cost) {
+  auto node = make_object<BuildResultNode>();
+  node->filename = std::move(filename);
+  node->args = std::move(args);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->time_cost = time_cost;
+  data_ = std::move(node);
+}
+
+MeasureResult::MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                             double timestamp) {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = std::move(costs);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  data_ = std::move(node);
+}
+
+MeasureResult MeasureResultNode::copy() const {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = costs;
+  node->error_no = error_no;
+  node->error_msg = error_msg;
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  return MeasureResult(node);
+}
+
+/********** LocalBuilder **********/
+LocalBuilder::LocalBuilder(int timeout, int n_parallel, const String& build_func) {
+  auto node = make_object<LocalBuilderNode>();
+  node->timeout = timeout;
+  node->n_parallel = n_parallel;
+  node->build_func = build_func;
+  data_ = std::move(node);
+}
+
+Array<BuildResult> LocalBuilderNode::Build(const Array<MeasureInput>& inputs, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_builder.build")) {
+    Array<BuildResult> results = (*f)(inputs, timeout, n_parallel, build_func, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_builder.build is not registered";
+  }
+  return Array<BuildResult>();
+}
+
+/********** LocalRunner **********/
+LocalRunner::LocalRunner(int timeout, int number, int repeat, int min_repeat_ms,
+                         double cooldown_interval) {
+  ObjectPtr<LocalRunnerNode> node = make_object<LocalRunnerNode>();
+  node->timeout = timeout;
+  node->number = number;
+  node->repeat = repeat;
+  node->min_repeat_ms = min_repeat_ms;
+  node->cooldown_interval = cooldown_interval;
+  data_ = std::move(node);
+}
+
+Array<MeasureResult> LocalRunnerNode::Run(const Array<MeasureInput>& inputs,
+                                          const Array<BuildResult>& build_results, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_runner.run")) {
+    Array<MeasureResult> results = (*f)(inputs, build_results, timeout, number, repeat,
+                                        min_repeat_ms, cooldown_interval, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_runner.run is not registered";
+  }
+  return Array<MeasureResult>();
+}
+
+/********** ProgramMeasurer **********/
+ProgramMeasurer::ProgramMeasurer(ProgramBuilder builder, ProgramRunner runner,
+                                 Array<MeasureCallback> callbacks, int verbose,
+                                 int max_continous_error) {
+  auto node = make_object<ProgramMeasurerNode>();
+  node->builder = std::move(builder);
+  node->runner = std::move(runner);
+  node->callbacks = std::move(callbacks);
+  node->verbose = verbose;
+  node->max_continous_error = max_continous_error < 0
+                                  ? ProgramMeasurerNode::DEFAULT_MAX_CONTINOUS_ERROR
+                                  : max_continous_error;
+  data_ = std::move(node);
+}
+
+void ProgramMeasurerNode::Reset() {
+  ct = error_ct = 0;
+  best_flops.clear();
+  best_ct.clear();
+  best_state.clear();
+}
+
+void ProgramMeasurerNode::Measure(const SearchTask& task, const SearchPolicy& policy,
+                                  const Array<MeasureInput>& inputs, Array<MeasureResult>* results,
+                                  int batch_size) {
+  results->clear();
+  results->reserve(inputs.size());
+
+  if (batch_size == -1) {
+    // set default batch size
+    batch_size = builder->n_parallel * 2;
+  }
+
+  StdCout(verbose) << "Get " << inputs.size() << " programs for measure. (This may take a while)"
+                   << std::endl;
+
+  for (size_t i = 0; i < inputs.size(); i += batch_size) {
+    Array<MeasureInput> input_batch(inputs.begin() + i,
+                                    inputs.begin() + std::min(i + batch_size, inputs.size()));
+    Array<MeasureResult> result_batch;
+
+    // build and run
+    SilentMeasure(task, input_batch, &result_batch);
+
+    // update current best state according to the new measure result
+    for (size_t j = 0; j < input_batch.size(); ++j) {
+      double flops;
+      if (result_batch[j]->error_no == 0) {
+        flops = task->compute_dag->flop_ct / FloatArrayMean(result_batch[j]->costs);
+        error_ct = 0;
+      } else {
+        flops = 0.0;
+        error_ct++;
+      }
+
+      const String& workload_key = input_batch[j]->task->workload_key;
+      if (flops > best_flops[workload_key]) {
+        best_flops[workload_key] = flops;
+        best_state[workload_key] = input_batch[j]->state;
+        best_ct[workload_key] = ct;
+      }
+
+      ct++;
+      StdCout(verbose) << std::fixed << std::setprecision(2) << Chars('=', 50) << "\n"
+                       << "No: " << ct << "\tGFLOPS: " << flops / 1e9 << " / "
+                       << best_flops[workload_key] / 1e9 << "\tresults: " << result_batch[j] << "\n"
+                       << Chars('=', 50) << "\n"
+                       << input_batch[j]->state << "\n";
+    }
+
+    // Call callback functions
+    for (const auto& callback : callbacks) {
+      callback->Callback(policy, input_batch, result_batch);
+    }
+
+    // Store result batch
+    for (auto& res : result_batch) {
+      results->push_back(res);
+    }
+
+    if (error_ct > max_continous_error) {
+      LOG(FATAL) << "Too many errors happened during tuning";
+    }
+  }
+}
+
+void ProgramMeasurerNode::SilentMeasure(const SearchTask& task, const Array<MeasureInput>& inputs,
+                                        Array<MeasureResult>* results) {
+  // Close the thread pool to avoid the conflits with python environment
+  ThreadPool::Global().Abort();
+
+  results->clear();
+  results->reserve(inputs.size());
+  Array<MeasureInput> input_batch(inputs.begin(), inputs.end());

Review comment:
       Cleaned.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }

Review comment:
       The index expression is integer, not *Float* operation

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }

Review comment:
       The index expression is integral, not *Float* operation

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }

Review comment:
       The index expression is integral, not **Float** operation




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }

Review comment:
       it's not part of flops imo




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages.push_back(stage);
+    UpdateStageAxis(stage, &stage_to_axes);
+  }
+
+  std::stringstream ss;
+  for (const auto& stage : stages) {
+    if (stage->op->IsInstance<te::ComputeOpNode>()) {
+      for (size_t i = 0; i < stage->leaf_iter_vars.size(); ++i) {
+        ss << stage->leaf_iter_vars[i]->var->name_hint;
+        if (i != stage->leaf_iter_vars.size() - 1) {
+          ss << ", ";
+        }
+      }
+      ss << " = "
+         << "tuple(" << stage->op->name << ".op.axis)"
+         << " + "
+         << "tuple(" << stage->op->name << ".op.reduce_axis)\n";
+    }
+  }
+  // Call each step's PrintAsPythonAPI method
+  for (const auto& step : transform_steps) {
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return ss.str();
+}
+
+State ComputeDAG::InferBound(const State& state) const {
+  State ret_state;
+  StateNode* pstate;
+
+  if (state->stages.size()) {

Review comment:
       nit
   ```suggestion
     if (!state->stages.empty()) {
   ```

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {

Review comment:
       nitpick, feel free to ignore:
   
   ```suggestion
   void UpdateStageToAxesMap(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
   ```

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages.push_back(stage);
+    UpdateStageAxis(stage, &stage_to_axes);
+  }
+
+  std::stringstream ss;
+  for (const auto& stage : stages) {
+    if (stage->op->IsInstance<te::ComputeOpNode>()) {
+      for (size_t i = 0; i < stage->leaf_iter_vars.size(); ++i) {
+        ss << stage->leaf_iter_vars[i]->var->name_hint;
+        if (i != stage->leaf_iter_vars.size() - 1) {
+          ss << ", ";
+        }
+      }
+      ss << " = "
+         << "tuple(" << stage->op->name << ".op.axis)"
+         << " + "
+         << "tuple(" << stage->op->name << ".op.reduce_axis)\n";
+    }
+  }
+  // Call each step's PrintAsPythonAPI method
+  for (const auto& step : transform_steps) {
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return ss.str();
+}
+
+State ComputeDAG::InferBound(const State& state) const {
+  State ret_state;
+  StateNode* pstate;
+
+  if (state->stages.size()) {
+    ret_state = state;
+    pstate = ret_state.CopyOnWrite();
+  } else {
+    // If the input state is incomplete with empty operation stage
+    // create a new state from init_state and update it first
+    ret_state = operator->()->init_state;
+    pstate = ret_state.CopyOnWrite();
+    pstate->transform_steps = state->transform_steps;
+    ret_state.DoSteps((*this));

Review comment:
       nit
   ```suggestion
       ret_state.DoSteps(*this);
   ```

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);

Review comment:
       For simplicity
   ```suggestion
       const te::Stage& stage = schedule[x];
   ```

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);

Review comment:
       same here
   ```suggestion
       const te::Stage& stage = schedule[x];
   ```

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");

Review comment:
       It is a bit hacky here to use env variables to pass information...It might be unimportant tho but I think we should set a global flag somewhere instead.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+ * a list of all related operations in topo order as well as a set of analyses over each operation
+ * stage (e.g. the total float operation count, consumer/producer relations of each operation
+ * stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+ * help the search policy to do some specific decisions during schedule search process.
+ *
+ * ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+ * (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+ * information get from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+  // TODO(merrymercy): Add more analyses later.
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the

Review comment:
       Improve the docstring by using my comments on the python file.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]

Review comment:
       I see. Looks like decoding workload key string includes 3 parts:
   1. Load the string in JSON format to be a list (e.g., `workload = ["matmul_ansor_test", 128, 128, 128]`).
   2. Get the workload name (e.g., `workload[0]`).
   3. Deserialize workload arguments (e.g., `deserialize_args(workload[1:])`).
   
   It seems better to have a separate function to perform all 3 steps so that we only see `name, args = decode(workload_key)`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):

Review comment:
       cc @merrymercy 




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});

Review comment:
       Really dumb question: I am not so sure if `ops.back()` suffices... Consider: A->B, A->C, in this case ops = {A, B, C}. Will `create_schedule({C})` make `B` ignored?




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {

Review comment:
       Hey I saw some overlap between TuningOptions and ProgramMeasurer, could you elaborate the difference and relationship?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).

Review comment:
       ```suggestion
       We convert a compute declaration described by `tvm.compute` (could be a single operator or a
       subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
       a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the total float operation count, consumer/producer relations of each operation 
       stage, whether an operation stage should be tiled/compute inlined ...). These analyses can
       help the search policy to make decisions during search process.
   
       ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
       (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
       information got from TVM schedule ...).
   ```
   
   
   
   Also, propagate the changes to c++ files.
   

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).

Review comment:
       Also, propagate the changes to c++ files.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.

Review comment:
       ```suggestion
           The ComputeDAG for the compute declaration.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target).
   Remove all "target" in before "compute declaration" in all files.
   Or replace "target" with "input" if you really want somthing before the noun.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
+1. We want fast incremental change to the loop structures;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+After the search is done, we will lower this IR to TVM IR with TVM's schedule primitives.
+Because we share a lot common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and the history steps to reach this state.
+
+    Each State corresponds to a specific schedule for the target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages

Review comment:
       The wrap implementation here is trying to make the State Python APIs similar to the existing TVM primitive APIs.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,470 @@
+# 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.
+
+"""
+Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling.
+"""

Review comment:
       Add the doc to address the confusion of builder.
   
   ```
   We separate the measurement into two steps: build and run.
   A builder builds the executable binary files and a runner runs the binary files to
   get the measurement results. The flow of data structures is 
   
                   `ProgramBuilder`                 `ProgramRunner`
   `MeasureInput` -----------------> `BuildResult` ----------------> `MeasureResult`
   
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       @tqchen Ok, I understand that(the stride argument has been set to 1 in default in utils.h), and it's fine for me the clean these code.
   Just confused about the "does not have to change now" above. :)




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):

Review comment:
       This follows the name convention in autotvm.
   We separate the measuremnet into two steps:
   build and run.
   So we have the flow `MeasureInput -> BuildResult -> MeasureResult`
   A builder takes `MeasureInput` and retusn `BuildResult`. A runner then takes `BuildResult` and returns `MeasureResult`




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ReplaySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add hash support in `struct hash<::tvm::ansor::Step>`. (search for this function in this file)
+ * 7. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class for a transformation step. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */
+  int stage_id;
+
+  static constexpr const char* _type_key = "ansor.Step";
+  TVM_DECLARE_BASE_OBJECT_INFO(StepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StepNode.
+ * \sa StepNode
+ */
+class Step : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(Step, ObjectRef, StepNode);
+};
+
+/*! \brief Reorder step that corresponds to te::Stage::reorder */
+class ReorderStepNode : public StepNode {
+ public:
+  /*!
+   * \brief The iterator ids after reorder.
+   * This array should specify the order of all iterators.
+   */
+  Array<PrimExpr> after_ids;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   */
+  void ApplyToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.ReorderStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ReorderStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ReorderStepNode.
+ * \sa ReorderStepNode
+ */
+class ReorderStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param after_ids The index of the iterators after reorder.
+   */
+  ReorderStep(int stage_id, const Array<PrimExpr>& after_ids);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(ReorderStep, Step, ReorderStepNode);
+};
+
+/*!
+ * \brief Split step that corresponds to te::Stage::split with additional
+ *  support of multiple-level of factors
+ */
+class SplitStepNode : public StepNode {
+ public:
+  /*! \brief The id of the iter to split. */
+  int iter_id;
+  /*! \brief The extent length of the axis to split. */
+  PrimExpr extent;
+  /*! \brief The split factors. */
+  Array<PrimExpr> lengths;
+  /*!
+   * \brief If true, the `lengths` denote the lengths of iterators
+   * from inner level to outer level
+   */
+  bool inner_to_outer;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return The iterator results after split.
+   */
+  Array<tir::IterVar> ApplyToSchedule(Array<te::Stage>* stages,
+                                      StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.SplitStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SplitStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SplitStepNode.
+ * \sa SplitStepNode
+ */
+class SplitStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param extent The index of the target iterator.
+   * \param lengths The extent length of the axis to split.
+   * \param inner_to_outer The split direction.
+   */
+  SplitStep(int stage_id, int iter_id, PrimExpr extent, const Array<PrimExpr>& lengths,
+            bool inner_to_outer);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(SplitStep, Step, SplitStepNode);
+};
+
+/*! \brief Fuse step that corresponds to te::Stage::fuse */
+class FuseStepNode : public StepNode {
+ public:
+  /*! \brief The ids of iterators to fuse. */
+  Array<PrimExpr> fused_ids;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return The iterator result after fuse.
+   */
+  tir::IterVar ApplyToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.FuseStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(FuseStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to FuseStepNode.
+ * \sa FuseStepNode
+ */
+class FuseStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param fused_ids The index of the target iterators to be fused.
+   */
+  FuseStep(int stage_id, const Array<PrimExpr>& fused_ids);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(FuseStep, Step, FuseStepNode);
+};
+
+}  // namespace ansor
+}  // namespace tvm
+
+// Hash and equal function for Step
+namespace std {
+
+/*! \brief The hash function of each transform step. */
+template <>
+struct hash<::tvm::ansor::Step> {

Review comment:
       This hash function is a historical issue, now removed.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief Compute declaration graph and its related analysis tools.
+ * ComputeDAG is also responsible for the interaction with the original TVM schedule system, to
+ * apply state to a runable TVM schedule or provide the schedule Python code.
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Computation declaration graph. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the
+   * equivalent `tvm::schedule`.
+   * \param transform_steps Transform steps of the target state.
+   * \return The return values can be used as arguments to `tvm.build` or `tvm.lower`.
+   */
+  std::pair<te::Schedule, Array<te::Tensor> > ApplySteps(const Array<Step>& transform_steps) const;
+  /*!
+   * \brief Print transform steps as equivalent python schedule API.
+   * \param transform_steps Transform steps of the target state.
+   * \return Python schedule code.
+   */
+  String PrintStepsAsPython(const Array<Step>& transform_steps) const;
+
+  /*!
+   * \brief Replay the transform steps and call ir_pass::InferBound to fill correct bound
+   * information.
+   * State api supports to define a split step with its split factor to be a blank placeholder,
+   * so sometimes we may get a State will incomplete iterator extent information.
+   * And another situation is after some steps (for exp. compute_at), it may be hard to track the
+   * extent change of all iterators.
+   * We perform infer bound using TVM schedule and fill the State with those informations. After
+   * applying this methods, the State is guaranteed to have complete interator extent information.
+   * \param transform_steps Transform steps of the target state.
+   * \return The State after inferbound.
+   */
+  State ReplayAndInferBound(const Array<Step>& transform_steps) const;
+  /*!
+   * \brief Fill the correct bound information for a given state by calling ir_pass::InferBound.
+   * \param state The target state.
+   * \return The State after inferbound.
+   */
+  State InferBound(const State& state) const;
+  /*!
+   * \brief Fill the correct bound information for a list of given states.
+   * Return the new states inplace.
+   * \param states A pointer to a State vector, States are updated inplace.
+   */
+  void InferBound(Array<State>* states) const;
+
+  TVM_DEFINE_OBJECT_REF_METHODS(ComputeDAG, ObjectRef, ComputeDAGNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(ComputeDAGNode);
+
+ private:
+  /*!
+   * \brief Internal common parts for replaying steps. This is the key method to apply steps to

Review comment:
       Deleted unnecessary functions.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""

Review comment:
       Also, propagate the changes to c++ files.

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.

Review comment:
       Remove all "target" before "StateObject", "State" and "ComputeDAG" in this file.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The target search task or workload key.
+    target : tvm.target.Target
+        The target device of this schedule search.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str] = 'default'
+        The search policy to be used for schedule search.
+    hardware_params : Optional[HardwareParams]
+        The hardware parameters of this schedule search.
+    tuning_options : Optional[TuningOptions]
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`

Review comment:
       ```suggestion
           A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`
   ```
   Remove all "target" before "te.Tensor", "tensor" in all files.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.

Review comment:
       ```suggestion
           The original ComputeDAG of this State.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});

Review comment:
       The whole code base is not tested with multiple outputs

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});

Review comment:
       The codebase has not been tested with multiple outputs.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);

Review comment:
       Is there specific rules why some of them are registered using `OBJECT_TYPE` but the others are `NODE_TYPE`? IIRC `NODE_TYPE` helps register creator into the vtable, so my understand is that if we don't customize our own creator, just use `NODE_TYPE` instead, and maybe define the VisitAttrs function?




----------------------------------------------------------------
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] jwfromm commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.

Review comment:
       The description of arguments for this class aren't useful. Instead of "the `arg` for the corresponding compute declaration" we should put some information about what that arg means. Here's one set of possible descriptions that would be more informative.
   
   ```
   Parameters
   ------------
   dag : ComputeDAG
       A graph representation of the compute declaration.
   workload_key : str
       A text identifier unique to this particular task.
   target : tvm.target.Target
       A TVM target that indicates the platform to generate code for.
   target_host : Optional[tvm.target.Target]
      Host compilation target, should be specified for targets such as `cuda` that require interaction with a host device.
   hardware_params : Optional[HardwareParams]
      A set of parameters that provide some basic information about the target hardware.
   ```

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The input function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):

Review comment:
       Is the plan that this will replace the `topi.util.get_const_int`? Seems like we have this same function in a few places in the code base already. It would be nice to consolidate. Same goes for other functions in this file.

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+        elif not isinstance(builder, tvm.ansor.measure.ProgramBuilder):
+            raise ValueError("Invalid builder: " + builder +
+                             " . TuningOptions expects a ProgramBuilder or string.")
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+        elif not isinstance(runner, tvm.ansor.measure.ProgramRunner):
+            raise ValueError("Invalid runner: " + runner +
+                             " . TuningOptions expects a ProgramRunner or string.")
+
+        measure_callbacks = measure_callbacks if measure_callbacks else []
+        pre_search_callbacks = pre_search_callbacks if pre_search_callbacks else []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',

Review comment:
       I'd argue that this function should require `task` to be a `SearchTask` as it simplifies the argument list by quite a bit and makes the behavior a little cleaner. Maybe there can be a separate helper function like `convert_workload` that creates `SearchTask`s from workload strings instead.

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is design to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[Builder, str] = 'local'
+      Builder which builds the program.
+    runner: Union[Runner, str] = 'local'
+      Runner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,

Review comment:
       I agree with @yangjunpro, this class is a fine way of collecting the tuning options, separating another dict out is messier.

##########
File path: python/tvm/ansor/record.py
##########
@@ -0,0 +1,157 @@
+# 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.
+
+""" Serialization and other I/O support for tuning logs (measurement records). """
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=None, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : Optional[int]
+            The maximum number of lines. None to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines if max_lines else -1,
+                                                      skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):

Review comment:
       given that this is already in the `ansor.record` module, we can probabily simplify the names of functions a little. For example I'd argue these reductions make for a cleaner API.
   
   `ansor.record.load_from_file` -> `ansor.record.load`
   `ansor.record.append_measure_records_to_file` -> `ansor.record.append_to_file`
   `ansor.record.best_measure_pair_in_file` -> `ansor.record.load_best`




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.
+    results: List[MeasureResults]
+        The target MeasureResults to be written.
+    """
+    _ffi_api.AppendMeasureRecordsToFile(filename, inputs, results)
+
+def best_measure_pair_in_file(filename, workload_key=None, target=None):
+    """ Return the best measurement pair form a log file. This may return none results if
+    there is no legal measure pair with the specified workload_key/target found from the log file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+    workload_key : Optional[str] = None
+        The workload key of the target compute declaration.

Review comment:
       ```suggestion
           The workload key of the compute declaration.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The generation function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    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.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the function.
+    """
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    if not func_name in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. "  % func,
+                         "Please register it with @ansor.register_workload")
+
+    args = serialize_args(args)
+
+    return json.dumps((func_name,) + args)
+
+
+def decode_workload_key_to_func_args(workload_key):
+    """ Decode a workload key to the registerd function name and its corresponding args.
+
+    Parameters
+    ----------
+    workload_key : str
+        The input workload key.
+
+    Returns
+    -------
+    name : str
+        The function name of this workload key.
+    args : List[Tensor]
+        The args of the generation function.
+    """
+    workload = json.loads(workload_key)
+    if not workload[0] in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. " % workload[0] +
+                         "Please register it with @ansor.register_workload")
+    return workload[0], deserialize_args(workload[1:])
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Get the input/output tensors from the workload key.
+
+    This method is usually used to create a ComputeDAG by workload key.
+
+    Parameters
+    ----------
+    workload_key : str
+        The input workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    name, args = decode_workload_key_to_func_args(workload_key)
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+    assert callable(lookup)
+    return lookup(*args)
+
+
+def save_workload_func_registry(filename):
+    """ Dump workload function registry to a pickle binary file.
+
+    Parameters
+    ----------
+    filename : str
+        The filename to dump workload function registry to.
+    """
+    global WORKLOAD_FUNC_REGISTRY

Review comment:
       Added `global` to all functions.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,103 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import tvm._ffi
+from tvm.runtime import Object
+from .loop_state import State, StateObject
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    tensors : List[Tensor]
+        `Tensor`s for a compute declaration.
+    """
+    def __init__(self, tensors):
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, tensors)
+
+    def get_init_state(self):

Review comment:
       Since the State object is in CopyOnWrite style, we'll store the initial state as an ObjecRef, and return the type cast result when calling this function.




----------------------------------------------------------------
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] yangjunpro commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    target : tvm.target.Target

Review comment:
       Is it better to pack the _target_, _target_host_ and _hardware_params_ as a single class(like TargetInfo?) holding the target related info since it looks that they are coherent. 

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):

Review comment:
       Yep, can you clarify your comment? @zhiics 
   It looks the API name is clear enough. 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:

Review comment:
       This is considered to be merged with ...

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The SearchTask or workload key for the computation declaration.
+    target : tvm.target.Target
+        The target device of this schedule search.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str] = 'default'
+        The search policy to be used for schedule search.
+    hardware_params : Optional[HardwareParams]
+        The hardware parameters of this schedule search.
+    tuning_options : Optional[TuningOptions]
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            # TODO(jcf94): This is an example policy for minimum system, will be upgrated to
+            # formal search policy later.
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    tuning_options = tuning_options if tuning_options else TuningOptions()

Review comment:
       _if tuning_options is not None_? 

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state.
+
+        The states can lose complete bound information after some transform steps
+        (e.g., compute_at).
+        We can call this function to infer and fill all the bound information.
+        This function calls TVM InferBound pass internally to get the bound.
+        The returned state of this function is guaranteed to have complete iterator extent
+        information.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        state : State
+            The State with complete bound information.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return State(_ffi_api.ComputeDAGInferBoundFromState(self, state_obj), self)
+
+    def __hash__(self):
+        # TODO(merrymercy): Implement this more carefully and move this to c++ as a member function

Review comment:
       The purpose of moving this into c++ is for performance? 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks

Review comment:
       ditto

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):

Review comment:
       ditto

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):

Review comment:
       When _builder_ is not str type,  shall we check whether it is a suitable builder related class? Such as LocalBuilder or something else. 

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):

Review comment:
       _gen_tvm_schedule_from_state()_ ? 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuningOptions, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tuning_options=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The SearchTask or workload key for the computation declaration.
+    target : tvm.target.Target
+        The target device of this schedule search.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str] = 'default'
+        The search policy to be used for schedule search.
+    hardware_params : Optional[HardwareParams]
+        The hardware parameters of this schedule search.
+    tuning_options : Optional[TuningOptions]
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            # TODO(jcf94): This is an example policy for minimum system, will be upgrated to
+            # formal search policy later.
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    tuning_options = tuning_options if tuning_options else TuningOptions()
+
+    if isinstance(task, str):
+        dag = ComputeDAG(task)
+        task = SearchTask(dag, task, target, target_host, hardware_params)
+    elif not isinstance(task, SearchTask):
+        raise ValueError("Invalid task: " + task +
+                         " . `ansor.auto_schedule` expects a `str` or `SearchTask`.")
+
+    sch, tensors = _ffi_api.AutoSchedule(task, search_policy, tuning_options)

Review comment:
       Should _target_ belong to _SearchTask_, or put it at the level of AutoSchedule itself? It looks that it is a global configuration for the execution of AutoSchedule run. 
   It looks that only _dag_ and _task_ work at the same semantic level. 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: int = -1
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[ProgramBuilder, str] = 'local'
+      ProgramBuilder which builds the program.
+    runner: Union[ProgramRunner, str] = 'local'
+      ProgramRunner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measurement.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks

Review comment:
       How about directly specifying the _measure_callbacks_ with default value _[]_? 

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer and fill the bound of all iterators of a state.
+
+        The states can lose complete bound information after some transform steps
+        (e.g., compute_at).
+        We can call this function to infer and fill all the bound information.
+        This function calls TVM InferBound pass internally to get the bound.
+        The returned state of this function is guaranteed to have complete iterator extent
+        information.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+        state : State
+            The State with complete bound information.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object

Review comment:
       This _if ... else.._ statment occurs quite a lot, I am wondering whether we can get rid of it to make code a little bit more compact? Or in which scenarios will we enter into different execution branch? 




----------------------------------------------------------------
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] tqchen commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Guys, please another look and https://tvm.apache.org/docs/contribute/code_review.html#approve-and-request-changes-explicitly


----------------------------------------------------------------
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] mbrookhart commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+ * a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+ * total float operation count, consumer/producer relations of each operation stage, whether an
+ * operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+ * to make decisions during search process.
+ * ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+ * (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+ * information got from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;

Review comment:
       Why not separate input and output tensors? This feels fragile.

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer bound for a state using TVM schedule.
+
+        State api supports to define a split step with its split factor to be a blank placeholder,
+        so sometimes we may get a State will incomplete iterator extent information.
+        And another situation is after some steps (for exp. compute_at), it may be hard to track
+        the extent change of all iterators.
+
+        We perform infer bound using TVM schedule and fill the State with those informations. After
+        applying this methods, the State is guaranteed to have complete interator extent
+        information.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        state : State
+            The State with complete bound information.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return State(_ffi_api.ComputeDAGInferBoundFromState(self, state_obj), self)
+
+    def __hash__(self):
+        # TODO(...): Implement this more carefully and move this to c++ as a member function

Review comment:
       Assign a name to the TODO

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,479 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+Array<te::Operation> TopoSortOps(const Array<te::Tensor>& tensors) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  Array<te::Operation> ops;
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops.reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops.push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+
+  return ops;
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {

Review comment:
       I'm a little uncomfortable using recursion here, are we should we wont overflow?

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,479 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+Array<te::Operation> TopoSortOps(const Array<te::Tensor>& tensors) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  Array<te::Operation> ops;
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops.reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops.push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+
+  return ops;
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};

Review comment:
       call this fail_ per Google Style Guide, especially with using it inside so many member methods.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.
+   * \param state The target State.
+   */
+  MeasureInput(SearchTask task, State state);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MeasureInput, ObjectRef, MeasureInputNode);
+};
+
+/*! \brief Store the result of a build. */
+class BuildResultNode : public Object {
+ public:
+  /*! \brief The filename of built binary file. */
+  std::string filename;

Review comment:
       Updated necessary std::vector to Array, std::string to String, some unordered_map to Map.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
+1. We want fast incremental change to the loop structures;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+After the search is done, we will lower this IR to TVM IR with TVM's schedule primitives.
+Because we share a lot common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")

Review comment:
       Yes, the State is used by SearchPolicy and the implementations are all in C++.
   Here we have State Python APIs is to prepare for providing custom search rule support in the future. Users can simply use Python to define their own search rules.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.

Review comment:
       ```suggestion
           The SearchTask.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,157 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    psutil = None

Review comment:
       Thanks for reminding this, I'll add that.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       @merrymercy Marisa means make the return type something like Optional<PrimExpr>




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.

Review comment:
       In my opinion, this seems more likly the explainations to the arg class. It may be better for users to get that from their own docs.




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.cc
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.cc
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step.
+ */
+
+#include "transform_step.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+/********** Reorder **********/
+ReorderStep::ReorderStep(int stage_id, const Array<PrimExpr>& after_ids) {
+  auto node = make_object<ReorderStepNode>();

Review comment:
       Shall we use `Array<Integer>` instead?

##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ReplaySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add hash support in `struct hash<::tvm::ansor::Step>`. (search for this function in this file)
+ * 7. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class for a transformation step. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */
+  int stage_id;
+
+  static constexpr const char* _type_key = "ansor.Step";
+  TVM_DECLARE_BASE_OBJECT_INFO(StepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StepNode.
+ * \sa StepNode
+ */
+class Step : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(Step, ObjectRef, StepNode);
+};
+
+/*! \brief Reorder step that corresponds to te::Stage::reorder */
+class ReorderStepNode : public StepNode {
+ public:
+  /*!
+   * \brief The iterator ids after reorder.
+   * This array should specify the order of all iterators.
+   */
+  Array<PrimExpr> after_ids;

Review comment:
       Array<Integer>

##########
File path: src/ansor/transform_step.cc
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.cc
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step.
+ */
+
+#include "transform_step.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+/********** Reorder **********/
+ReorderStep::ReorderStep(int stage_id, const Array<PrimExpr>& after_ids) {
+  auto node = make_object<ReorderStepNode>();
+  node->stage_id = stage_id;
+  for (const auto& x : after_ids) {
+    CHECK(x.defined() && x->IsInstance<IntImmNode>());
+  }
+  node->after_ids = after_ids;
+  data_ = std::move(node);
+}
+
+void ReorderStepNode::ApplyToSchedule(Array<te::Stage>* stages,
+                                      StageToAxesMap* stage_to_axes) const {
+  auto stage = (*stages)[stage_id];
+  const Array<IterVar>& axes = stage_to_axes->at(stage);
+  CHECK_EQ(after_ids.size(), axes.size());
+
+  Array<IterVar> new_axes;
+  new_axes.reserve(axes.size());
+  for (auto i : after_ids) {
+    new_axes.push_back(axes[i.as<IntImmNode>()->value]);
+  }
+  stage.reorder(new_axes);
+
+  stage_to_axes->Set(stage, std::move(new_axes));
+  stages->Set(stage_id, std::move(stage));
+}
+
+String ReorderStepNode::PrintAsPythonAPI(Array<te::Stage>* stages,
+                                         StageToAxesMap* stage_to_axes) const {
+  const auto& stage = (*stages)[stage_id];
+  std::stringstream ss;
+
+  ss << "s[" << CleanName(stage->op->name) << "].reorder(";
+  for (size_t i = 0; i < after_ids.size(); ++i) {
+    ss << CleanName((*stage_to_axes)[stage][after_ids[i].as<IntImmNode>()->value]->var->name_hint);
+    if (i != after_ids.size() - 1) {
+      ss << ", ";
+    }
+  }
+  ss << ")\n";
+
+  ApplyToSchedule(stages, stage_to_axes);
+  return ss.str();
+}
+
+/********** Split **********/
+Array<IterVar> ApplySplitToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes,
+                                    int stage_id, int iter_id, const Array<PrimExpr>& lengths,
+                                    bool inner_to_outer) {
+  auto stage = (*stages)[stage_id];
+  const Array<IterVar>& axes = stage_to_axes->at(stage);
+
+  Array<IterVar> outs;
+  if (inner_to_outer) {
+    IterVar outer = axes[iter_id], inner;
+    for (int i = static_cast<int>(lengths.size()) - 1; i >= 0; i--) {
+      IterVar to_split = outer;
+      stage.split(to_split, lengths[i], &outer, &inner);
+      outs.push_back(inner);
+    }
+    outs.push_back(outer);
+  } else {
+    IterVar outer, inner = axes[iter_id];
+    for (size_t i = 0; i < lengths.size(); i++) {
+      IterVar to_split = inner;
+      stage.split_by_nparts(to_split, lengths[i], &outer, &inner);
+      outs.push_back(outer);
+    }
+    outs.push_back(inner);
+  }
+
+  Array<IterVar> new_axes;
+  new_axes.insert(new_axes.end(), axes.begin(), axes.begin() + iter_id);
+  if (inner_to_outer) {
+    for (auto x = outs.rbegin(); x != outs.rend(); ++x) {
+      new_axes.push_back((*x));
+    }
+  } else {
+    for (const auto& x : outs) {
+      new_axes.push_back(x);
+    }
+  }
+  new_axes.insert(new_axes.end(), axes.begin() + iter_id + 1, axes.end());
+
+  stage_to_axes->Set(stage, std::move(new_axes));
+  stages->Set(stage_id, std::move(stage));
+  return outs;
+}
+
+String PrintSplitAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes, int stage_id,
+                             int iter_id, const Array<PrimExpr>& lengths, bool inner_to_outer) {
+  const auto& stage = (*stages)[stage_id];
+  auto to_split = stage_to_axes->at(stage)[iter_id];
+  const auto& func_name = CleanName(stage->op->name);
+  const auto& outs =
+      ApplySplitToSchedule(stages, stage_to_axes, stage_id, iter_id, lengths, inner_to_outer);
+  CHECK_EQ(outs.size(), lengths.size() + 1);
+
+  std::stringstream ss;
+  int size = static_cast<int>(lengths.size());
+  if (inner_to_outer) {
+    for (int i = size - 1; i >= 0; i--) {
+      ss << CleanName(outs[size - i]->var->name_hint) << ", "
+         << CleanName(outs[size - i - 1]->var->name_hint) << " = s[" << func_name << "].split("
+         << CleanName(to_split->var->name_hint) << ", factor=" << lengths[i] << ")\n";
+      to_split = outs[size - i];
+    }
+  } else {
+    for (int i = 0; i < size; i++) {
+      ss << CleanName(outs[i]->var->name_hint) << ", " << CleanName(outs[i + 1]->var->name_hint)
+         << " = s[" << func_name << "].split(" << CleanName(to_split->var->name_hint)
+         << ", nparts=" << lengths[i] << ")\n";
+      to_split = outs[i + 1];
+    }
+  }
+
+  return ss.str();
+}
+
+SplitStep::SplitStep(int stage_id, int iter_id, PrimExpr extent, const Array<PrimExpr>& lengths,
+                     bool inner_to_outer) {
+  auto node = make_object<SplitStepNode>();
+  node->stage_id = stage_id;
+  // Extent can be a unreducible expression in some special cases
+  if (extent->IsInstance<IntImmNode>()) {
+    node->extent = std::move(extent);
+  }
+  node->iter_id = iter_id;
+  node->lengths = lengths;
+  node->inner_to_outer = inner_to_outer;
+  data_ = std::move(node);
+}
+
+Array<IterVar> SplitStepNode::ApplyToSchedule(Array<te::Stage>* stages,
+                                              StageToAxesMap* stage_to_axes) const {
+  return ApplySplitToSchedule(stages, stage_to_axes, stage_id, iter_id, lengths, inner_to_outer);
+}
+
+String SplitStepNode::PrintAsPythonAPI(Array<te::Stage>* stages,
+                                       StageToAxesMap* stage_to_axes) const {
+  return PrintSplitAsPythonAPI(stages, stage_to_axes, stage_id, iter_id, lengths, inner_to_outer);
+}
+
+/********** Fuse **********/
+FuseStep::FuseStep(int stage_id, const Array<PrimExpr>& fused_ids) {

Review comment:
       Array<Integer>




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {

Review comment:
       In design, TuningOptions is a interface structure(without any member function) that passes some user control options to search policy, while the ProgramMeasurer is responsible for building and measuring a schedule(is has Measure() function).




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */

Review comment:
       I think I must missed something. Shall we also have `kThreadZ` and `kBlockZ`?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {

Review comment:
       In design, TuningOptions is a interface structure(without any member function) that passes some user control options to search policy, while the ProgramMeasurer is responsible for building and measuring a schedule(it has Measure() function).




----------------------------------------------------------------
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] yangjunpro edited a comment on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   > @jcf94 and @merrymercy thanks for all the hard work! Can I request that we put another unresolved issue? In my opinion the written English parts i.e comments, explanations, etc could still use some improvement with both content and grammar and I would propose in general that we do some at least 1 or 2 rounds of full documentation polish (comments, examples, tests, tutorials, etc) before we officially release a feature (in this case when all of Ansor is landed in master). We tried to do this with Relay but I think we should continue to strive to do a better job with new features like this.
   
   I do appreciate and support the proposal. Let's move forward with the feature upstream process and after the major features being merged into master we can work together to refine the documentation. 


----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;

Review comment:
       “Optional” makes the “nullable” semantics more explicit. To convert an “Optional” to its not-nullable type, use “.value()” :-)




----------------------------------------------------------------
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] electriclilies commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):

Review comment:
       I think the name is confusing too, I would suggest print_transform_history_from_state, or print_python_history_from_state, since the function prints the transform history in python.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});

Review comment:
       Hmmm let’s make it correct 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] yangjunpro commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_by_func(func):
+    """ Register a workload by generation function.

Review comment:
       So in the comments "generator function" may be less ambiguous:)

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The generation function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key(func, args):
+    """ make a workload key from function and arguments.
+
+    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.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the function.
+    """
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    if not func_name in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. "  % func,
+                         "Please register it with @ansor.register_workload")
+
+    args = serialize_args(args)
+
+    return json.dumps((func_name,) + args)
+
+
+def decode_workload_key_to_func_args(workload_key):
+    """ Decode a workload key to the registerd function name and its corresponding args.
+
+    Parameters
+    ----------
+    workload_key : str
+        The input workload key.
+
+    Returns
+    -------
+    name : str
+        The function name of this workload key.
+    args : List[Tensor]
+        The args of the generation function.
+    """
+    workload = json.loads(workload_key)
+    if not workload[0] in WORKLOAD_FUNC_REGISTRY:
+        raise ValueError("%s is not registered. " % workload[0] +
+                         "Please register it with @ansor.register_workload")
+    return workload[0], deserialize_args(workload[1:])
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Get the input/output tensors from the workload key.
+
+    This method is usually used to create a ComputeDAG by workload key.
+
+    Parameters
+    ----------
+    workload_key : str
+        The input workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    name, args = decode_workload_key_to_func_args(workload_key)
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+    assert callable(lookup)
+    return lookup(*args)
+
+
+def save_workload_func_registry(filename):
+    """ Dump workload function registry to a pickle binary file.
+
+    Parameters
+    ----------
+    filename : str
+        The filename to dump workload function registry to.
+    """
+    global WORKLOAD_FUNC_REGISTRY

Review comment:
       Why do we only have a explicit _global_ here? 

##########
File path: src/ansor/auto_schedule.cc
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.cc
+ * \brief The user interface of the Ansor auto-scheduler.
+ */
+
+#include "auto_schedule.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(TuningOptionsNode);
+
+TuningOptions::TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round,
+                             int verbose, ProgramBuilder builder, ProgramRunner runner,
+                             Array<MeasureCallback> measure_callbacks,
+                             Array<SearchCallback> pre_search_callbacks) {
+  auto node = make_object<TuningOptionsNode>();

Review comment:
       Why not directly assign _data__ with _make_object<TuningOptionsNode>()_? 

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a computation graph).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The generation function that returns the compute declaration Tensors.
+
+    Examples
+    --------
+    @ansor.register_workload
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    assert callable(func)
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+
+    WORKLOAD_FUNC_REGISTRY[func_name] = func

Review comment:
       Do we need to explicitly add a _global WORKLOAD_FUNC_REGISTRY_ statement?

##########
File path: src/ansor/auto_schedule.cc
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.cc
+ * \brief The user interface of the Ansor auto-scheduler.
+ */
+
+#include "auto_schedule.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(TuningOptionsNode);
+
+TuningOptions::TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round,
+                             int verbose, ProgramBuilder builder, ProgramRunner runner,
+                             Array<MeasureCallback> measure_callbacks,
+                             Array<SearchCallback> pre_search_callbacks) {
+  auto node = make_object<TuningOptionsNode>();
+  node->num_measure_trials = num_measure_trials;
+  node->early_stopping = early_stopping;
+  node->num_measures_per_round = num_measures_per_round;
+  node->verbose = verbose;
+  node->builder = std::move(builder);
+  node->runner = std::move(runner);
+  node->measure_callbacks = std::move(measure_callbacks);
+  node->pre_search_callbacks = std::move(pre_search_callbacks);
+  data_ = std::move(node);
+}
+
+std::pair<te::Schedule, Array<te::Tensor> > AutoSchedule(SearchTask task,
+                                                         SearchPolicy search_policy,
+                                                         TuningOptions tuning_options) {
+  // Create a ProgramMeasurer to handle the schedule build and performance measure
+  ProgramMeasurer measurer =
+      ProgramMeasurer(tuning_options->builder, tuning_options->runner,
+                      tuning_options->measure_callbacks, tuning_options->verbose);
+  // Search for the best schedule
+  State state = search_policy->Search(
+      task, tuning_options->num_measure_trials, tuning_options->early_stopping,

Review comment:
       I think one reasons is that _tuniong_options_ contains more what  ProgramMeasure constructor needs, so they choose to expose as little as possible between different modules? 
   
   I am also wondering whether it is better to pack those lengthy things into a single holder to avoid the bulky  argument pass-in.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.

Review comment:
       This is just for debug and pretty printing




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.

Review comment:
       ```suggestion
           This is used for debugging.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int

Review comment:
       Thanks for pointing out and it makes sense to me. We can add a TODO and revisit this 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 a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("stages", &stages);
+    v->Visit("transform_steps", &transform_steps);
+    v->Visit("complete", &complete);
+  }
+
+  static constexpr const char* _type_key = "ansor.State";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StateNode, Object);
+
+ private:
+  /*!
+   * \brief The up-to-date ComputeDAG of this state, used for some steps that may change the
+   * stage structure of the ComputeDAG, for exp. CacheReadStep/CacheWriteStep(Will be added later).
+   * The default value is an empty ObjectRef. (means no modification to the original DAG)
+   */
+  ObjectRef current_compute_dag;
+};
+
+/*!
+ * \brief Managed reference to StateNode.
+ * \sa StateNode
+ */
+class State : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param ops `te::Operation`s for a compute declaration.
+   */
+  explicit State(const Array<te::Operation>& ops);
+
+  /*!
+   * \brief Print the state to a human readable string.
+   * \param delete_trivial_loop True for skipping the trivial loops.
+   * (undefined or extent == 1, default set to True)
+   * \return The human readable state structure.
+   */
+  String ToStr(bool delete_trivial_loop = true) const;
+
+  /*!
+   * \brief General do step functions with a runtime dynamic dispatcher.
+   * \param dag The target ComputeDAG.

Review comment:
       ```suggestion
      * \param dag The input ComputeDAG.
   ```

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.
+ */
+class StateNode : public Object {
+ public:
+  /*! \brief Current stages and loop structures. */
+  Array<Stage> stages;
+  /*! \brief History transformation steps. */
+  Array<Step> transform_steps;
+  /*! \brief Indicate whether this state has unfilled tile sizes. */
+  bool complete;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("stages", &stages);
+    v->Visit("transform_steps", &transform_steps);
+    v->Visit("complete", &complete);
+  }
+
+  static constexpr const char* _type_key = "ansor.State";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StateNode, Object);
+
+ private:
+  /*!
+   * \brief The up-to-date ComputeDAG of this state, used for some steps that may change the
+   * stage structure of the ComputeDAG, for exp. CacheReadStep/CacheWriteStep(Will be added later).
+   * The default value is an empty ObjectRef. (means no modification to the original DAG)
+   */
+  ObjectRef current_compute_dag;
+};
+
+/*!
+ * \brief Managed reference to StateNode.
+ * \sa StateNode
+ */
+class State : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param ops `te::Operation`s for a compute declaration.
+   */
+  explicit State(const Array<te::Operation>& ops);
+
+  /*!
+   * \brief Print the state to a human readable string.
+   * \param delete_trivial_loop True for skipping the trivial loops.
+   * (undefined or extent == 1, default set to True)
+   * \return The human readable state structure.
+   */
+  String ToStr(bool delete_trivial_loop = true) const;
+
+  /*!
+   * \brief General do step functions with a runtime dynamic dispatcher.
+   * \param dag The target ComputeDAG.
+   */
+  void DoSteps(const ComputeDAG& dag);
+
+  /* Step APIs for State. */
+
+  /*!
+   * \brief Schedule primitive corresponds to te.reorder.
+   * \param stage_id The index of the target stage.
+   * \param order The target iterator order.
+   */
+  void reorder(int stage_id, const Array<Iterator>& order);
+  /*!
+   * \brief Schedule primitive corresponds to te.split.
+   * \param stage_id The index of the target stage.
+   * \param it The target iterator.
+   * \param lengths The target split factors. Can be None to be filled by search policy.
+   * \param inner_to_outer True for split from inner to outer & False for outer to inner.
+   * \return The iterator results after split.
+   */
+  Array<Iterator> split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                        bool inner_to_outer = true);
+  /*!
+   * \brief Schedule primitive corresponds to te.fuse.
+   * \param stage_id The index of the target stage.
+   * \param iters The target iterators to be fused.
+   * \return The iterator result after fuse.
+   */
+  Iterator fuse(int stage_id, const Array<Iterator>& iters);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(State, ObjectRef, StateNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StateNode);
+
+ private:
+  /* Do transform steps
+   * Note: The following functions only change loop state but do not change transform_history.
+   * We separate these functions out, so you can call them for replay easily given history steps */
+
+  /*!
+   * \brief Apply reorder step to current state.
+   * \param step A ReorderStep.
+   */
+  void DoReorderStep(const ReorderStep& step);
+  /*!
+   * \brief Apply split step to current state.
+   * \param step A SplitStep.
+   * \return The iterator results after split.
+   */
+  Array<Iterator> DoSplitStep(const SplitStep& step);
+  /*!
+   * \brief Apply fuse step to current state.
+   * \param step A FuseStep.
+   * \return The iterator result after fuse.
+   */
+  Iterator DoFuseStep(const FuseStep& step);
+
+  /*!
+   * \brief Common function for DoSplitStep and DoFollowSplitStep(Will be added later).
+   * \param stage_id The index of the target stage.
+   * \param iter_id The index of the target iterator.
+   * \param lengths The target split factors.

Review comment:
       remove target

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The target range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Stage, ObjectRef, StageNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(StageNode);
+};
+
+/*!
+ * \brief A state in the search process.
+ * It consists of the current loop structure and a history of transformations used to construct it.
+ * Each State corresponds to a specific schedule for its target ComputeDAG.

Review comment:
       ```suggestion
    * Each State corresponds to a specific schedule for its ComputeDAG.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       Yes, we should just remove the thread pool




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+
+    if callable(lookup):
+        args = deserialize_args(workload[1:])
+        return lookup(*args)
+    return lookup
+
+
+@ tvm._ffi.register_func("ansor.workload_key_to_dag")
+def workload_key_to_dag(workload_key):
+    """ Decode a workload key to a compute dag.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    dag : ComputeDAG
+        ComputeDAG to the registered compute declaration.
+    """
+    tensors = workload_key_to_tensors(workload_key)
+    return ComputeDAG(tensors)
+
+
+def make_workload_key_func(func, args):

Review comment:
       Renamed `make_workload_key_func` to `make_workload_key_by_func`.
   Remove the unused one for 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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/__init__.py
##########
@@ -0,0 +1,34 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       That also works if we all agree, we can send a followup PR for 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    if not GLOBAL_BUILD_ARGUMENTS:
+        raise ValueError("GLOBAL_BUILD_ARGUMENTS not found")
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=broad-except
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=broad-except
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose == 1:
+            if error_no == MeasureErrorNo.NO_ERROR:
+                print(".", end="")
+            else:
+                print(".E", end="")  # Build error
+        return filename, args, error_no, error_msg, time.time() - tic
+
+    res = call_func_with_timeout(timeout, timed_func)
+    if isinstance(res, TimeoutError):
+        if verbose == 1:
+            print(".T", end="")  # Build timeout
+        res = None, [], MeasureErrorNo.BUILD_TIMEOUT, None, timeout
+
+    return res
+
+
+@tvm._ffi.register_func("ansor.local_builder.build")
+def local_builder_build(inputs, timeout, n_parallel, build_func, verbose):
+    """ Local builder build function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    global GLOBAL_BUILD_ARGUMENTS
+    GLOBAL_BUILD_ARGUMENTS = (inputs, build_func, timeout, verbose)

Review comment:
       `multiprocessing.Pool` is the default multiprocessing library in python.
   There is no way to work around this.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief Hardware related parameters */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+  /*! \brief The max length of an axis to be unrolled or vectorized. */
+  int max_unroll_vec;
+  /*! \brief The max split factor for the innermost tile. */
+  int max_innermost_split_factor;
+
+  // Limitation params for GPU

Review comment:
       I can get the meaning of this sentence...but doesn't feel right...may be "Parameters limited to GPU"? Let's find some native speaker...

##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ReplaySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add hash support in `struct hash<::tvm::ansor::Step>`. (search for this function in this file)
+ * 7. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class for a transformation step. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */
+  int stage_id;
+
+  static constexpr const char* _type_key = "ansor.Step";
+  TVM_DECLARE_BASE_OBJECT_INFO(StepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StepNode.
+ * \sa StepNode
+ */
+class Step : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(Step, ObjectRef, StepNode);
+};
+
+/*! \brief Reorder step that corresponds to te::Stage::reorder */
+class ReorderStepNode : public StepNode {
+ public:
+  /*!
+   * \brief The iterator ids after reorder.
+   * This array should specify the order of all iterators.
+   */
+  Array<PrimExpr> after_ids;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   */
+  void ApplyToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.ReorderStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ReorderStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ReorderStepNode.
+ * \sa ReorderStepNode
+ */
+class ReorderStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param after_ids The index of the iterators after reorder.
+   */
+  ReorderStep(int stage_id, const Array<PrimExpr>& after_ids);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(ReorderStep, Step, ReorderStepNode);
+};
+
+/*!
+ * \brief Split step that corresponds to te::Stage::split with additional
+ *  support of multiple-level of factors
+ */
+class SplitStepNode : public StepNode {
+ public:
+  /*! \brief The id of the iter to split. */
+  int iter_id;
+  /*! \brief The extent length of the axis to split. */
+  PrimExpr extent;
+  /*! \brief The split factors. */
+  Array<PrimExpr> lengths;
+  /*!
+   * \brief If true, the `lengths` denote the lengths of iterators
+   * from inner level to outer level
+   */
+  bool inner_to_outer;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return The iterator results after split.
+   */
+  Array<tir::IterVar> ApplyToSchedule(Array<te::Stage>* stages,
+                                      StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.SplitStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SplitStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SplitStepNode.
+ * \sa SplitStepNode
+ */
+class SplitStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param extent The index of the target iterator.
+   * \param lengths The extent length of the axis to split.
+   * \param inner_to_outer The split direction.
+   */
+  SplitStep(int stage_id, int iter_id, PrimExpr extent, const Array<PrimExpr>& lengths,
+            bool inner_to_outer);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(SplitStep, Step, SplitStepNode);
+};
+
+/*! \brief Fuse step that corresponds to te::Stage::fuse */
+class FuseStepNode : public StepNode {
+ public:
+  /*! \brief The ids of iterators to fuse. */
+  Array<PrimExpr> fused_ids;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return The iterator result after fuse.
+   */
+  tir::IterVar ApplyToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.FuseStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(FuseStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to FuseStepNode.
+ * \sa FuseStepNode
+ */
+class FuseStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param fused_ids The index of the target iterators to be fused.
+   */
+  FuseStep(int stage_id, const Array<PrimExpr>& fused_ids);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(FuseStep, Step, FuseStepNode);
+};
+
+}  // namespace ansor
+}  // namespace tvm
+
+// Hash and equal function for Step
+namespace std {
+
+/*! \brief The hash function of each transform step. */
+template <>
+struct hash<::tvm::ansor::Step> {

Review comment:
       a really dumb question, why not we split it into 3 hash functions separately...? If we want to take hash of `ansor::Step` we can just runtime dispatch to one of those 3.

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+  /*! \brief The max length of an axis to be unrolled or vectorized. */
+  int max_unroll_vec;
+  /*! \brief The max split factor for the innermost tile. */
+  int max_innermost_split_factor;

Review comment:
       What is this used for btw? Like max tiling size?




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.h
##########
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_AUTO_SCHEDULE_UTILS_H_
+#define TVM_AUTO_SCHEDULE_UTILS_H_
+
+#include <dmlc/common.h>
+#include <tvm/tir/expr.h>
+
+#include <algorithm>
+#include <deque>
+#include <exception>
+#include <future>
+#include <string>
+#include <thread>
+#include <tuple>
+#include <utility>
+#include <vector>
+
+namespace std {
+
+/*! \brief Hash function for std::pair */
+template <typename T1, typename T2>
+struct hash<std::pair<T1, T2>> {
+  std::size_t operator()(const std::pair<T1, T2>& k) const {
+    return ::dmlc::HashCombine(std::hash<T1>()(k.first), std::hash<T2>()(k.second));
+  }
+};
+
+/*! \brief Hash function for std::tuple */
+template <typename T1, typename T2, typename T3>
+struct hash<std::tuple<T1, T2, T3>> {
+  std::size_t operator()(const std::tuple<T1, T2, T3>& k) const {
+    return ::dmlc::HashCombine(
+        ::dmlc::HashCombine(std::hash<T1>()(std::get<0>(k)), std::hash<T2>()(std::get<1>(k))),
+        std::hash<T3>()(std::get<2>(k)));
+  }
+};
+
+}  // namespace std
+
+namespace tvm {
+namespace auto_schedule {
+
+/********** Utilities for Array, std::string **********/
+/*! \brief Get the first appearance index of elements in an Array */
+template <typename T>
+inline void GetIndices(const Array<T>& array, const Array<T>& to_locate, Array<Integer>* indices) {
+  for (const auto& v : to_locate) {
+    auto it = std::find(array.begin(), array.end(), v);
+    if (it != array.end()) {
+      indices->push_back(it - array.begin());
+    } else {
+      LOG(FATAL) << "Cannot find the item";
+    }
+  }
+}
+
+/*! \brief Get the first appearance index of an element in an Array */
+template <typename T>
+inline int GetIndex(const Array<T>& array, const T& to_locate) {
+  for (size_t i = 0; i < array.size(); ++i) {
+    if (array[i] == to_locate) {
+      return i;
+    }
+  }
+  LOG(FATAL) << "Cannot find the item";
+  return -1;
+}
+
+/*! \brief Replace a sub-string to another sub-string in a string */
+inline void StrReplace(std::string* base, const std::string& from, const std::string& to) {
+  auto pos = base->find(from);
+  while (pos != std::string::npos) {
+    base->replace(pos, from.size(), to);
+    pos = base->find(from, pos + to.size());
+  }
+}
+
+/********** Utilities for TVM Containers / ByteArray **********/
+/*! \brief Compute mean of a FloatImm array */
+inline double FloatArrayMean(const Array<PrimExpr>& float_array) {
+  double sum = 0;
+  if (float_array.empty()) {
+    return 0.0;
+  }
+
+  for (const auto& x : float_array) {
+    auto floatimm = x.as<tir::FloatImmNode>();
+    CHECK(floatimm != nullptr);
+    sum += floatimm->value;
+  }
+  return sum / float_array.size();
+}
+
+/********** Other Utilities **********/
+/*! \brief Get an int value from an Expr */
+inline int64_t GetIntImm(const PrimExpr& expr) {
+  auto pint = expr.as<IntImmNode>();
+  CHECK(pint != nullptr);
+  return pint->value;
+}
+
+/*! \brief Compute the product of the lengths of axes */
+inline int64_t AxisLengthProd(const Array<tir::IterVar>& axes) {
+  int64_t ret = 1.0;
+  for (const auto& x : axes) {
+    if (const IntImmNode* imm = x->dom->extent.as<IntImmNode>()) {
+      ret *= imm->value;
+    } else {
+      return -1.0;
+    }
+  }
+  return ret;
+}
+
+/*!
+ * \brief Clean the name of an iterator to make it valid in python code.
+ * \param str The original name.
+ * \return The cleaned name.
+ */
+inline std::string CleanName(const std::string& str) {
+  std::string ret = str;
+  StrReplace(&ret, ".", "_");
+  StrReplace(&ret, "@", "_");
+  StrReplace(&ret, "outer", "o");
+  StrReplace(&ret, "inner", "i");
+  return ret;
+}
+
+/*! \brief An empty output stream */
+class NullStream : public std::ostream {
+ public:
+  NullStream() : std::ostream(nullptr) {}
+  NullStream(const NullStream&) : std::ostream(nullptr) {}
+  static NullStream& Global();
+};
+
+template <class T>
+NullStream& operator<<(NullStream& os, const T& value) {
+  return os;
+}
+
+/*! \brief Get std cout with verbose control */
+inline std::ostream& StdCout(int verbose, int setting = 1) {
+  return verbose >= setting ? std::cout : NullStream::Global();
+}
+
+/*! \brief Print multiple chars */
+inline std::string Chars(const char& str, int times) {
+  std::stringstream ret;
+  for (int i = 0; i < times; ++i) {
+    ret << str;
+  }
+  return ret.str();
+}
+
+/*! \brief Print a title */
+inline void PrintTitle(const std::string& title, int verbose) {
+  StdCout(verbose) << Chars('-', 60) << "\n"
+                   << Chars('-', 25) << "  [ " << title << " ]\n"
+                   << Chars('-', 60) << std::endl;
+}
+
+/*!
+ * \brief A simple thread pool.
+ * TODO(merrymercy): Move this to `src/support/parallel_for`
+ */
+class ThreadPool {

Review comment:
       Change theb use of ThreadPool to parallel_for




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.

Review comment:
       Remove all "target"  before "SearchTask", "search task", "compute declaration". Replace it with "input" if you really want something before the noun.




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {

Review comment:
       Let us change the namespace to `auto_schedule`, so that the module can be a generic module of tvm.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):

Review comment:
       Add a TODO below.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters

Review comment:
       The `max_unroll_vec` & `max_innermost_split_factor` does more likely to be user setting values rather than the hardware parameters.
   Since they're not used in this minimum system, remove them for now, maybe we can add them to the parameter of SearchPolicy?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover auto schedule generation for different
+ * ops/subgraphs, and in the meantime, a custom rule mechanism will be provided to enable
+ * user-defined template search. (which should play a same role as the current AutoTVM template)
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search and returns the best schedule get.
+ * 2. Imformations about the target ops/subgraphs can be acquired from SearchTask, this structure
+ * also contains HardwareParams which can be used to limit the search space. (For exp. limit the
+ * max vectorize size depending on the vector unit weight of a specific device)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to SearchPolicyNode.
+   */
+  virtual void Callback(SearchPolicyNode* policy) = 0;
+
+  static constexpr const char* _type_key = "ansor.SearchCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchCallbackNode.
+ * \sa SearchCallbackNode
+ */
+class SearchCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(SearchCallback, ObjectRef, SearchCallbackNode);
+};
+
+/*!
+ * \brief The base class for search policy.
+ */
+class SearchPolicyNode : public Object {
+ public:
+  /*! \brief The current search task. */
+  SearchTask cur_task;
+  /*!
+   * \brief Verbose level to control the screen output during schedule search.
+   * 0 for silent, 1 to output information.
+   */
+  int verbose;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("cur_task", &cur_task);
+    v->Visit("verbose", &verbose);
+  }
+
+  /*!
+   * \brief Do schedule search for a task. Takes the SearchTask as input and returns the best state
+   * get during the search process.
+   * \param task The target search task.
+   * \param num_measure_trials Total schedules to be tried during this search.
+   * \param early_stopping Early stop if no better schedule is found.
+   * \param num_measures_per_round Max measure batch in one search round.
+   * \param verbose Verbose level. 0 for silent, 1 to output information during schedule search.
+   * \param measurer A ProgramMeasurer which packs Builder & Runner inside.
+   * \param pre_search_callbacks SearchCallback to be called before schedule search.
+   * \return The best state get.
+   */
+  virtual State Search(SearchTask task, int num_measure_trials, int early_stopping,
+                       int num_measures_per_round, int verbose, ProgramMeasurer measurer,
+                       Array<SearchCallback> pre_search_callbacks) = 0;
+
+  /*!
+   * \brief Call SearchCallback with the current SearchPolicyNode
+   * \param callbacks SearchCallback to be called.
+   */
+  void RunCallbacks(const Array<SearchCallback>& callbacks);
+
+  static constexpr const char* _type_key = "ansor.SearchPolicy";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchPolicyNode, Object);
+
+ protected:
+  /*!
+   * \brief The set of already measured states.
+   * We store the string format for redundancy check.

Review comment:
       Doc updated.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,200 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    target : tvm.target.Target

Review comment:
       This will be refactored after this  https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1

Review comment:
       There are many verbosity levels so we should use int.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TE and TE's schedule primitives
+ * - loop_state.cc:     How each step reflect on LoopState

Review comment:
       ```suggestion
    * - transform_step.cc: How each step interacts with TE and TE's schedule primitives
    * - loop_state.cc:     How each step updates LoopState
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters

Review comment:
       We can just set values like 32 or 64 for them. The search algorithm is not sensitive to them and the default values work well  for most platforms including Intel CPUs and ARM CPUs.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);

Review comment:
       In my understanding, `NODE_TYPE` == `OBJECT_TYPE` + `REFLECTION_VTABLE`, so we registered some classes that might need class member relection to `NODE_TYPE` and left the others to `OBJECT_TYPE`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):

Review comment:
       Since this is not used in Ansor minimum system, remove this function for now.
   Rename `register_workload_func` to `register_workload_by_func`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):

Review comment:
       @zhiics We already have pretty printing for state. This is pretty-printing for steps.
   @MarisaKirisame This is not codegen. The printed string is for debugging.
   
   This function prints python code, I am happy with the current name.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The input function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : Union[tvm.tir.expr, int]
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, expr.IntImm):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, expr.IntImm):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : Tuple[tvm.tir.expr]
+        The input.
+
+    Returns
+    -------
+    out_tuple : Tuple[int]
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret

Review comment:
       Thanks! We does have planed to merge AutoTVM & Ansor to one system.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});

Review comment:
       Added a TODO here to check multiple outputs in the future, currently just left it here to use `ops.back()`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):

Review comment:
       You can see the docstring below.
   We use fork to copy arguments from a global variable.
   This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
   




----------------------------------------------------------------
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] electriclilies commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):

Review comment:
       I agree, the description should explain what LocalBuilder is and its relationship the the Builder thread pool. 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/__init__.py
##########
@@ -0,0 +1,34 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       I think @MarisaKirisame means the namespace should be a noun. So auto_scheduler is better.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,223 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a schedule for its ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to manipulate the current state.
+The transform history is a sequence of `TransformStep` which will finally be mapped to TVM schedule
+primitives. The steps can also be used for the serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures. The search policy needs to get the
+immediate loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """ A stage in the compute declaration. Similar to tvm.te.schedule.Stage. """
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a list of transformation steps used to construct it.
+
+    Each State corresponds to a specific schedule for its ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The StateObject corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages

Review comment:
       Hey I think I got confused too with this cache. Would love to hear elaboration :-) What is this cache used for? If this is not performance critical, IMHO in python we don't need a cache; if it is perf critical, we should deal with it in CXX if possible.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.

Review comment:
       Doc updated.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.

Review comment:
       ```suggestion
           The ComputeDAG for the compute declaration.
   ```
   Do not overuse `target` because `target` already has its meaning (i.e. the hardware target).
   Remove all "target" before "compute declaration" in all files.
   Or replace "target" with "input" if you really want something before the noun.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       ?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+  /*! \brief The max length of an axis to be unrolled or vectorized. */
+  int max_unroll_vec;
+  /*! \brief The max split factor for the innermost tile. */
+  int max_innermost_split_factor;

Review comment:
       Removed for 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] comaniac commented on pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   > You guys used crossover, and it often suck in genetic algorithm. Had you guys done ablation study of genetic algorithm with pure-mutation approach?
   
   Crossover has to be used carefully as you pointed out. We currently only cross stages to avoid invalid schedules, so it is impactful on multi-stage workloads such as Winograd conv2d. Anyways, crossover would be added at the later stage of up streaming Ansor, so we don't need to worry about that in this PR 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] jcf94 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.cc
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.cc
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step.
+ */
+
+#include "transform_step.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+/********** Reorder **********/
+ReorderStep::ReorderStep(int stage_id, const Array<PrimExpr>& after_ids) {
+  auto node = make_object<ReorderStepNode>();

Review comment:
       Updated PrimExpr ot Integer.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""

Review comment:
       cc @merrymercy 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       yes
   




----------------------------------------------------------------
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] jroesch commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial

Review comment:
       ```suggestion
   In `Program sampling`, we use predefined precise or heuristic rules to generate several initial
   ```
   I think it makes sense to clarify that prefined means exact or precise rules here.

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will

Review comment:
       ```suggestion
   Candidate schedules are measured against the specific hardware target.
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:

Review comment:
       We should mark this TODO with someone responsible for it imo. 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model

Review comment:
       ```suggestion
   schedules. Based on these initial starting points, we perform `Performance Tuning` which uses evolutionary search based on a cost model to select schedules with the best performance. 
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is design to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[Builder, str] = 'local'
+      Builder which builds the program.
+    runner: Union[Runner, str] = 'local'
+      Runner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,

Review comment:
       yeah I agree, there are lots of fields here and its a bit hard to consume

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):

Review comment:
       This name reads a bit awkward in english imo, perhaps we could clarify? `TuningConfig`, `TuningOptions`? I think plurality is important here given we have more then one option. 

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """

Review comment:
       ```suggestion
   """ The Ansor computational graph and related program analyses. """
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
+1. We want fast incremental change to the loop structures;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+After the search is done, we will lower this IR to TVM IR with TVM's schedule primitives.
+Because we share a lot common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")

Review comment:
       Why have a split here? it seems like all the state should be in C++ cc @tqchen 

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.

Review comment:
       ```suggestion
    *  other supporting data structures.
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):

Review comment:
       It is not clear what the purpose of this function is, we should also not pass arguments globally. 

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and

Review comment:
       ```suggestion
    * \brief The base class for search policy, including the abstract definition of search policy and
   ```

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover auto schedule generation for different
+ * ops/subgraphs, and in the meantime, a custom rule mechanism will be provided to enable
+ * user-defined template search. (which should play a same role as the current AutoTVM template)
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search and returns the best schedule get.
+ * 2. Imformations about the target ops/subgraphs can be acquired from SearchTask, this structure
+ * also contains HardwareParams which can be used to limit the search space. (For exp. limit the
+ * max vectorize size depending on the vector unit weight of a specific device)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to SearchPolicyNode.

Review comment:
       ```suggestion
      * \param policy A pointer to a SearchPolicyNode.
   ```

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.

Review comment:
       The writing here could be improved/clarified. 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.

Review comment:
       Can you clarify here, meta-information is vague and non-informative to me 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of

Review comment:
       ```suggestion
       """ The parameters of target hardware used to guide the search process of
   ```

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.

Review comment:
       Can we maybe add some explanation of how this is different then the many other DAGs that exist in TVM today. I think a common user failure mode is confusion between the various IRs and representations. 

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):

Review comment:
       @zhiics can you clarify? 

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function

Review comment:
       I think we should avoid any kind of global configuration, and instead unify configuration into context options like we use universally throughout the system cc @tqchen 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """

Review comment:
       ```suggestion
       """ The base class of search policies. """
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
+1. We want fast incremental change to the loop structures;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+After the search is done, we will lower this IR to TVM IR with TVM's schedule primitives.

Review comment:
       ```suggestion
   When the search is complete, we will lower this IR to TVM IR with TVM's schedule primitives.
   ```

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[Builder, str] = 'local'
+      Builder which builds the program.
+    runner: Union[Runner, str] = 'local'
+      Runner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid runner: " + runner)
+
+        measure_callbacks = [] if measure_callbacks is None else measure_callbacks
+        pre_search_callbacks = [] if pre_search_callbacks is None else pre_search_callbacks
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, num_measure_trials, early_stopping, num_measures_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(task, target, target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The task parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    task : Union[SearchTask, str]
+        The target search task or workload key.
+    target : tvm.target.Target
+        The target device of this schedule search.
+    target_host : Optional[tvm.target.Target]
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str] = 'default'
+        The search policy to be used for schedule search.
+    hardware_params : Optional[HardwareParams]
+        The hardware parameters of this schedule search.
+    tune_option : Optional[TuneOption]
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            # TODO(jcf94): This is an example policy for minimum system, will be upgrated to
+            # formal search policy later.
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    tune_option = tune_option if tune_option else TuneOption()
+
+    if isinstance(task, str):
+        dag = ComputeDAG(task)
+        task = SearchTask(dag, task, target, target_host, hardware_params)
+    elif not isinstance(task, SearchTask):
+        raise ValueError("Invalid task: " + task + ". Expect a string or SearchTask")

Review comment:
       ```suggestion
           raise ValueError("Invalid task: " + task + " . `ansor.auto_schedule` expects a `str` or `SearchTask`.")
   ```

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:

Review comment:
       It might be better to combine these two paragraphs into one, that more clearly states "We have designed a new loop IR specifically for schedule search. Our loop IR is similar to TVM's TIR but importantly has three important additions: ...."

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop

Review comment:
       `flexible` doesn't mean anything in a technical setting imo, it would be good to clarify why we need a new IR here.

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer bound for a state using TVM schedule.

Review comment:
       I believe I know what you are talking about, but `infer bound` is relatively vague and is worth explaining more clearly to users what this is doing. 

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):

Review comment:
       yeah I agree, I also personally think `build` is nonsensical in a compiler and we should strive to remove it from TVM period as it carries no technical meaning in the compiler world, and is vague enough to everyone else it doesn't actually let us know what operation is occurring. 

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.

Review comment:
       ```suggestion
   and a history of transformations used to construct it. 
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_by_func(func):
+    """ Register a workload by generation function.

Review comment:
       This doesn't make sense to me, can you explain? 

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""

Review comment:
       ```suggestion
   """Common utilities for ansor."""
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):

Review comment:
       See above comment. 

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,211 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search. A state consists a current loop structure
+and the transform history to reach its current loop structure.
+To enable flexible manipulation of the loop structures, we implemented a lightweight loop
+structure IR (Intermediate Representation) based on the original TVM IR but specifically
+for schedule search.
+
+We don't use the existing TVM IR but to extend a new Sketch IR on it is because:
+1. We want fast incremental change to the loop structures;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+After the search is done, we will lower this IR to TVM IR with TVM's schedule primitives.
+Because we share a lot common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and the history steps to reach this state.
+
+    Each State corresponds to a specific schedule for the target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages

Review comment:
       its not clear to me that the cache should be in the Python interface, it seems like almost all operations on this object are just interacting with cache then C++.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):

Review comment:
       ```suggestion
   class ProgramBuilder(Object):
   ```
   We should be more specific here

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.

Review comment:
       If we are using a serialization format we should version and track it, unversioned serialization is a bug waiting to happen imo. 

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):

Review comment:
       👍 

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):

Review comment:
       A better question is do we need classes for this at all? could we not just use reader/writer functions? cc @tqchen 

##########
File path: src/ansor/serialization.h
##########
@@ -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.
+ */
+
+/*!
+ * \file ansor/serialization.h
+ * \brief Json serialization format for dumping and loading tuning records.
+ */
+
+#ifndef TVM_ANSOR_SERIALIZATION_H_
+#define TVM_ANSOR_SERIALIZATION_H_
+
+#include <fstream>
+#include <string>
+#include <utility>
+
+#include "measure.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Callback for logging the input and results of measurements to file */

Review comment:
       RE: my Python comment it is probably more flexible to just have reader/writer callbacks. 

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       Can we improve this comment? 

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,507 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;

Review comment:
       Improve comments in the file, vague comments are more likely to bitrot or mislead the reader of the code. 

##########
File path: src/ansor/utils.cc
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;

Review comment:
       Yes that would be good. cc @tqchen having multiple thread pools scares me a bit thoughts? 

##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */

Review comment:
       Same comment as above I placed on the Python class name. 

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""

Review comment:
       I like using `record` or `configuration` or something that is different. To me we should use this as a way to move away from `log file` terminology, its incredibly confusing when interacting with people outside of TVM as they assume you are talking about traditional service logging. It would be good to have a distinct name for the results of AutoTVM/Ansor imo. 

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief Hardware related parameters */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+  /*! \brief The max length of an axis to be unrolled or vectorized. */
+  int max_unroll_vec;
+  /*! \brief The max split factor for the innermost tile. */
+  int max_innermost_split_factor;
+
+  // Limitation params for GPU
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_unroll_vec", &max_unroll_vec);
+    v->Visit("max_innermost_split_factor", &max_innermost_split_factor);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   * \param max_unroll_vec The max length of an axis to be unrolled or vectorized.
+   * \param max_innermost_split_factor The max split factor for the innermost tile.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes, int max_unroll_vec,
+                 int max_innermost_split_factor);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*! \brief Meta-info for a search task */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */
+  ComputeDAG compute_dag;
+  /*! \brief The workload key for target compute declaration. */
+  String workload_key;
+  /*! \brief The target device of this search task. */
+  Target target;
+  /*! \brief The target host device of this search task. */
+  Target target_host;
+  /*! \brief Hardware parameters used in this search task. */
+  HardwareParams hardware_params;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("compute_dag", &compute_dag);
+    v->Visit("workload_key", &workload_key);
+    v->Visit("target", &target);
+    v->Visit("target_host", &target_host);
+    v->Visit("hardware_params", &hardware_params);
+  }
+
+  static constexpr const char* _type_key = "ansor.SearchTask";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SearchTaskNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchTaskNode.
+ * \sa SearchTaskNode
+ */
+class SearchTask : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param compute_dag The ComputeDAG for target compute declaration.

Review comment:
       This comment isn't clear "target compute declaration"

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover auto schedule generation for different
+ * ops/subgraphs, and in the meantime, a custom rule mechanism will be provided to enable
+ * user-defined template search. (which should play a same role as the current AutoTVM template)
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search and returns the best schedule get.
+ * 2. Imformations about the target ops/subgraphs can be acquired from SearchTask, this structure
+ * also contains HardwareParams which can be used to limit the search space. (For exp. limit the
+ * max vectorize size depending on the vector unit weight of a specific device)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to SearchPolicyNode.
+   */
+  virtual void Callback(SearchPolicyNode* policy) = 0;
+
+  static constexpr const char* _type_key = "ansor.SearchCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchCallbackNode.
+ * \sa SearchCallbackNode
+ */
+class SearchCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(SearchCallback, ObjectRef, SearchCallbackNode);
+};
+
+/*!
+ * \brief The base class for search policy.
+ */
+class SearchPolicyNode : public Object {
+ public:
+  /*! \brief The current search task. */
+  SearchTask cur_task;
+  /*!
+   * \brief Verbose level to control the screen output during schedule search.
+   * 0 for silent, 1 to output information.
+   */
+  int verbose;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("cur_task", &cur_task);
+    v->Visit("verbose", &verbose);
+  }
+
+  /*!
+   * \brief Do schedule search for a task. Takes the SearchTask as input and returns the best state
+   * get during the search process.
+   * \param task The target search task.
+   * \param num_measure_trials Total schedules to be tried during this search.
+   * \param early_stopping Early stop if no better schedule is found.
+   * \param num_measures_per_round Max measure batch in one search round.
+   * \param verbose Verbose level. 0 for silent, 1 to output information during schedule search.
+   * \param measurer A ProgramMeasurer which packs Builder & Runner inside.
+   * \param pre_search_callbacks SearchCallback to be called before schedule search.
+   * \return The best state get.
+   */
+  virtual State Search(SearchTask task, int num_measure_trials, int early_stopping,
+                       int num_measures_per_round, int verbose, ProgramMeasurer measurer,
+                       Array<SearchCallback> pre_search_callbacks) = 0;
+
+  /*!
+   * \brief Call SearchCallback with the current SearchPolicyNode
+   * \param callbacks SearchCallback to be called.
+   */
+  void RunCallbacks(const Array<SearchCallback>& callbacks);
+
+  static constexpr const char* _type_key = "ansor.SearchPolicy";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchPolicyNode, Object);
+
+ protected:
+  /*!
+   * \brief The set of already measured states.
+   * We store the string format for redundancy check.

Review comment:
       More details please. 

##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief Compute declaration graph and its related analysis tools.
+ * ComputeDAG is also responsible for the interaction with the original TVM schedule system, to
+ * apply state to a runable TVM schedule or provide the schedule Python code.
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Computation declaration graph. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the
+   * equivalent `tvm::schedule`.
+   * \param transform_steps Transform steps of the target state.
+   * \return The return values can be used as arguments to `tvm.build` or `tvm.lower`.
+   */
+  std::pair<te::Schedule, Array<te::Tensor> > ApplySteps(const Array<Step>& transform_steps) const;
+  /*!
+   * \brief Print transform steps as equivalent python schedule API.
+   * \param transform_steps Transform steps of the target state.
+   * \return Python schedule code.
+   */
+  String PrintStepsAsPython(const Array<Step>& transform_steps) const;
+
+  /*!
+   * \brief Replay the transform steps and call ir_pass::InferBound to fill correct bound
+   * information.
+   * State api supports to define a split step with its split factor to be a blank placeholder,
+   * so sometimes we may get a State will incomplete iterator extent information.
+   * And another situation is after some steps (for exp. compute_at), it may be hard to track the
+   * extent change of all iterators.
+   * We perform infer bound using TVM schedule and fill the State with those informations. After
+   * applying this methods, the State is guaranteed to have complete interator extent information.
+   * \param transform_steps Transform steps of the target state.
+   * \return The State after inferbound.
+   */
+  State ReplayAndInferBound(const Array<Step>& transform_steps) const;
+  /*!
+   * \brief Fill the correct bound information for a given state by calling ir_pass::InferBound.
+   * \param state The target state.
+   * \return The State after inferbound.
+   */
+  State InferBound(const State& state) const;
+  /*!
+   * \brief Fill the correct bound information for a list of given states.
+   * Return the new states inplace.
+   * \param states A pointer to a State vector, States are updated inplace.
+   */
+  void InferBound(Array<State>* states) const;
+
+  TVM_DEFINE_OBJECT_REF_METHODS(ComputeDAG, ObjectRef, ComputeDAGNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(ComputeDAGNode);
+
+ private:
+  /*!
+   * \brief Internal common parts for replaying steps. This is the key method to apply steps to

Review comment:
       Can you clarify why this is split out, vs ReplayAndInferBounds, does it need to be interface?

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief Hardware related parameters */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+  /*! \brief The max length of an axis to be unrolled or vectorized. */
+  int max_unroll_vec;
+  /*! \brief The max split factor for the innermost tile. */
+  int max_innermost_split_factor;
+
+  // Limitation params for GPU
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_unroll_vec", &max_unroll_vec);
+    v->Visit("max_innermost_split_factor", &max_innermost_split_factor);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   * \param max_unroll_vec The max length of an axis to be unrolled or vectorized.
+   * \param max_innermost_split_factor The max split factor for the innermost tile.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes, int max_unroll_vec,
+                 int max_innermost_split_factor);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*! \brief Meta-info for a search task */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */
+  ComputeDAG compute_dag;
+  /*! \brief The workload key for target compute declaration. */
+  String workload_key;
+  /*! \brief The target device of this search task. */
+  Target target;
+  /*! \brief The target host device of this search task. */
+  Target target_host;
+  /*! \brief Hardware parameters used in this search task. */
+  HardwareParams hardware_params;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("compute_dag", &compute_dag);
+    v->Visit("workload_key", &workload_key);
+    v->Visit("target", &target);
+    v->Visit("target_host", &target_host);
+    v->Visit("hardware_params", &hardware_params);
+  }
+
+  static constexpr const char* _type_key = "ansor.SearchTask";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SearchTaskNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchTaskNode.

Review comment:
       ```suggestion
    * \brief Managed reference to a SearchTaskNode.
   ```

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class for search policy, including the abstract defination of search policy and
+ * some other supporting structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+ * schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+ * and evolutionary search to seek for schedules with the best performance. Candidate schedules
+ * will be measured in the target hardware.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover auto schedule generation for different
+ * ops/subgraphs, and in the meantime, a custom rule mechanism will be provided to enable
+ * user-defined template search. (which should play a same role as the current AutoTVM template)
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search and returns the best schedule get.
+ * 2. Imformations about the target ops/subgraphs can be acquired from SearchTask, this structure
+ * also contains HardwareParams which can be used to limit the search space. (For exp. limit the
+ * max vectorize size depending on the vector unit weight of a specific device)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to SearchPolicyNode.
+   */
+  virtual void Callback(SearchPolicyNode* policy) = 0;
+
+  static constexpr const char* _type_key = "ansor.SearchCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchCallbackNode.
+ * \sa SearchCallbackNode
+ */
+class SearchCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(SearchCallback, ObjectRef, SearchCallbackNode);
+};
+
+/*!
+ * \brief The base class for search policy.
+ */
+class SearchPolicyNode : public Object {
+ public:
+  /*! \brief The current search task. */
+  SearchTask cur_task;
+  /*!
+   * \brief Verbose level to control the screen output during schedule search.
+   * 0 for silent, 1 to output information.
+   */
+  int verbose;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("cur_task", &cur_task);
+    v->Visit("verbose", &verbose);
+  }
+
+  /*!
+   * \brief Do schedule search for a task. Takes the SearchTask as input and returns the best state
+   * get during the search process.
+   * \param task The target search task.
+   * \param num_measure_trials Total schedules to be tried during this search.
+   * \param early_stopping Early stop if no better schedule is found.
+   * \param num_measures_per_round Max measure batch in one search round.
+   * \param verbose Verbose level. 0 for silent, 1 to output information during schedule search.
+   * \param measurer A ProgramMeasurer which packs Builder & Runner inside.
+   * \param pre_search_callbacks SearchCallback to be called before schedule search.
+   * \return The best state get.
+   */
+  virtual State Search(SearchTask task, int num_measure_trials, int early_stopping,
+                       int num_measures_per_round, int verbose, ProgramMeasurer measurer,
+                       Array<SearchCallback> pre_search_callbacks) = 0;
+
+  /*!
+   * \brief Call SearchCallback with the current SearchPolicyNode
+   * \param callbacks SearchCallback to be called.
+   */
+  void RunCallbacks(const Array<SearchCallback>& callbacks);
+
+  static constexpr const char* _type_key = "ansor.SearchPolicy";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchPolicyNode, Object);
+
+ protected:
+  /*!
+   * \brief The set of already measured states.
+   * We store the string format for redundancy check.
+   */
+  std::unordered_set<String> measured_states_set_;
+  /*! \brief The array of already measured states. */
+  std::vector<State> measured_states_vector_;
+  /*! \brief The throughputs of already measured states */
+  std::vector<float> measured_states_throughputs_;
+};
+
+/*!
+ * \brief Managed reference to SearchPolicyNode.

Review comment:
       ```suggestion
    * \brief Managed reference to a SearchPolicyNode.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGPrintPythonCodeFromState(self, state_obj)
+
+    def infer_bound_from_state(self, state):
+        """
+        Infer bound for a state using TVM schedule.

Review comment:
       Updated the explanation.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       To avoid involving extra review effort, removed ThreadPool from the current code base. cc @tqchen 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.

Review comment:
       ```suggestion
           The function that returns the compute declaration Tensors.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");

Review comment:
       Oh, this is a hack for some ablation study in the paper. We should delete this.
   @jcf94 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).

Review comment:
       Also, propagate the changes to c++ files.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,162 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_by_func(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @ansor.register_workload_by_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def make_workload_key_by_func(func, args):

Review comment:
       Renamed to `make_workload_key`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/measure.py
##########
@@ -0,0 +1,480 @@
+# 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.
+
+"""
+Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We separate the measurement into two steps: build and run.
+A builder builds the executable binary files and a runner runs the binary files to
+get the measurement results. The flow of data structures is
+
+                `ProgramBuilder`                 `ProgramRunner`
+`MeasureInput` -----------------> `BuildResult` ----------------> `MeasureResult`
+
+We implement these in python to utilize python's multiprocessing and error handling.
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# We use fork and a global variable to copy arguments between processings.
+# This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("auto_schedule.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("auto_schedule.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The SearchTask of this measure.
+    state : State
+        The State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("auto_schedule.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("auto_schedule.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("auto_schedule.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=True):

Review comment:
       All verbosity levels should be int because we can have multiple levels.

##########
File path: python/tvm/auto_schedule/measure_record.py
##########
@@ -0,0 +1,157 @@
+# 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.
+
+""" Serialization and other I/O support for tuning logs (measurement records). """

Review comment:
       ```suggestion
   """ Serialization and other I/O support for measurement records (tuning logs). """
   ```

##########
File path: python/tvm/auto_schedule/auto_schedule.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""
+User interface for TVM Auto-scheduler.
+
+The basic schedule search process for TVM Auto-scheduler is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("auto_schedule.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("auto_schedule.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("auto_schedule.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("auto_schedule.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("auto_schedule.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.

Review comment:
       ```suggestion
         With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
         measurement. This can be used to get a runnable schedule quickly without auto-tuning.
   ```

##########
File path: python/tvm/auto_schedule/auto_schedule.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""
+User interface for TVM Auto-scheduler.
+
+The basic schedule search process for TVM Auto-scheduler is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("auto_schedule.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("auto_schedule.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.
+    workload_key : str
+        The workload key for the corresponding compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("auto_schedule.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("auto_schedule.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("auto_schedule.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of measurement trials.
+      The search policy measures `num_measure_trials` schedules in total and returns the best one
+      among them.
+      With `num_measure_trials` == 0, the policy will do the schedule search but won't involve
+      measurement.
+      This can be used to get a runnable schedule quickly without auto-tuning.
+    early_stopping: Optional[int]
+      Stop the tuning early if getting no improvement after n measurements.
+    num_measures_per_round: int = 64
+      The number of schedules to be measured at each search round.
+      The whole schedule search process will try a total number of `num_measure_trials` in several
+      rounds.
+    verbose: boolean = True
+      Verbosity level. False for silent, True to output information during schedule search.

Review comment:
       This should be int because we can have many levels.
   The previous comment from @MarisaKirisame  is wrong.

##########
File path: src/auto_schedule/measure.h
##########
@@ -0,0 +1,438 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * The flow of data structures is MeasureInput -> BuildeResult -> MeasureResult.

Review comment:
       Copy the comments from the python file to replace these lines

##########
File path: src/auto_schedule/auto_schedule.h
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/auto_schedule.h
+ * \brief The user interface of the TVM Auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_AUTO_SCHEDULE_AUTO_SCHEDULE_H_
+#define TVM_AUTO_SCHEDULE_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*!
+   * \brief Verbosity level.
+   * False for silent, true to output information during schedule searching.
+   */
+  bool verbose;

Review comment:
       should be int




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Thank all the reviewers! Most comments from the reviewers are addressed.
   
   We summarize the unresolved comments here. They will be addressed by follow-up PRs. We want to merge this PR first to accelerate the upstream process.
   - The design of `python/tvm/auto_schedule/auto_scheduler.py::auto_schedule`. This is the most important user interface to use the auto-scheduler. We will send an RFC to discuss this.
   - Move `src/auto_schedule/utils.h::ThreadPool` to `src/support/parallle_for.h` (create a new file)
   - Consolidate the usage of `python/tvm/auto_schedule/utils.py::get_const_int`, `python/tvm/auto_schedule/utils.py::get_const_tuple` in the whole TVM code base.
   
   @jroesch @jwfromm @yangjunpro @MarisaKirisame @junrushao1994  Please take another look and approve.


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).

Review comment:
       Please propagate the changes to c++ files.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.
+ *
+ * We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+ * subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+ * a list of all related operations in topo order as well as a set of analyses over each operation
+ * stage (e.g. the total float operation count, consumer/producer relations of each operation
+ * stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+ * help the search policy to do some specific decisions during schedule search process.
+ *
+ * ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+ * (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+ * information get from TVM schedule ...).
+ */
+
+#ifndef TVM_ANSOR_COMPUTE_DAG_H_
+#define TVM_ANSOR_COMPUTE_DAG_H_
+
+#include <tvm/te/schedule.h>
+
+#include <utility>
+
+#include "loop_state.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief The Ansor computational graph and related program analyses. */
+class ComputeDAGNode : public Object {
+ public:
+  /*! \brief Input and output tensors. */
+  Array<te::Tensor> tensors;
+  /*! \brief All related operations in topo order. */
+  Array<te::Operation> ops;
+  /*! \brief Number of total float operations for this ComputeDAG. */
+  double flop_ct;
+  /*! \brief The initial state without any transform steps. */
+  State init_state;
+  // TODO(merrymercy): Add more analyses later.
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("tensors", &tensors);
+    v->Visit("ops", &ops);
+    v->Visit("flop_ct", &flop_ct);
+    v->Visit("init_state", &init_state);
+  }
+
+  static constexpr const char* _type_key = "ansor.ComputeDAG";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ComputeDAGNode.
+ * \sa ComputeDAGNode
+ */
+class ComputeDAG : public ObjectRef {
+ public:
+  /*! \brief The constructor.
+   * \param tensors `te::Tensor`s for a compute declaration.
+   */
+  explicit ComputeDAG(Array<te::Tensor> tensors);
+
+  /*!
+   * \brief Apply transform steps to the init state of this DAG, and get the
+   * equivalent `tvm::schedule`.
+   * \param transform_steps Transform steps of the target state.
+   * \param stages A pointer to a `te::Stage` Array, default to be nullptr.
+   * Pass a valid pointer if these information needs to be used outside this function.
+   * \param stage_to_axes A pointer to a StageToAxesMap, default to be nullptr.
+   * Pass a valid pointer if these information needs to be used outside this function.
+   * \return The return values can be used as arguments to `tvm.build` or `tvm.lower`.
+   */
+  std::pair<te::Schedule, Array<te::Tensor> > ApplySteps(
+      const Array<Step>& transform_steps, Array<te::Stage>* stages = nullptr,
+      StageToAxesMap* stage_to_axes = nullptr) const;
+
+  /*!
+   * \brief Print transform steps as equivalent python schedule API.
+   * \param transform_steps Transform steps of the target state.

Review comment:
       You can safely delete all "target" in your docstring in this file. It is redundant.
   Replace it with "input" if you really want something before the noun.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // GPU related parameters got from device query API
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);

Review comment:
       Maybe move to `HardwareParams` instead, given right now we prefer constructor-like stuff in the container?

##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during schedule searching. */
+  int verbose;
+  /*! \brief ProgramBuilder which builds the program */
+  ProgramBuilder builder;
+  /*! \brief ProgramRunner which runs the program and measure time costs */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback functions to be called after each measure batch */
+  Array<MeasureCallback> measure_callbacks;
+  /*! \brief SearchCallback functions to be called before schedule search */
+  Array<SearchCallback> pre_search_callbacks;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_measure_trials", &num_measure_trials);
+    v->Visit("early_stopping", &early_stopping);
+    v->Visit("num_measures_per_round", &num_measures_per_round);
+    v->Visit("verbose", &verbose);
+    v->Visit("builder", &builder);
+    v->Visit("runner", &runner);
+    v->Visit("measure_callbacks", &measure_callbacks);
+    v->Visit("pre_search_callbacks", &pre_search_callbacks);
+  }
+
+  static constexpr const char* _type_key = "ansor.TuningOptions";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningOptionsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to TuningOptionsNode.
+ * \sa TuningOptionsNode
+ */
+class TuningOptions : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor
+   * \param num_measure_trials Number of total measurement trials.
+   * \param early_stopping Stops early the tuning if no improvement after n measurements.
+   * \param num_measures_per_round The number of programs to be measured at each search round.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during schedule
+   * search.
+   * \param builder ProgramBuilder which builds the program.
+   * \param runner ProgramRunner which runs the program and measure time costs.
+   * \param measure_callbacks MeasureCallback functions to be called after each measure batch.
+   * \param pre_search_callbacks SearchCallback functions to be called before schedule search.
+   */
+  TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round, int verbose,
+                ProgramBuilder builder, ProgramRunner runner,
+                Array<MeasureCallback> measure_callbacks,
+                Array<SearchCallback> pre_search_callbacks);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TuningOptions, ObjectRef, TuningOptionsNode);
+};
+
+/*!
+ * \brief Auto schedule search for a given compute declaration.
+ * \param task The search task of the compute declaration.
+ * \param search_policy The search policy to be used for schedule search.
+ * \param tuning_options Tuning and measurement options.
+ * \return A `te::schedule` and the a Array of `te::Tensor` to be used in `tvm.lower` or
+ * `tvm.build`.
+ */
+std::pair<te::Schedule, Array<te::Tensor>> AutoSchedule(SearchTask task, SearchPolicy search_policy,

Review comment:
       Let's prepend `TVM_DLL` as it directly interacts with FFI.
   
   ```suggestion
   TVM_DLL std::pair<te::Schedule, Array<te::Tensor>> AutoSchedule(SearchTask task, SearchPolicy search_policy,
   ```

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {

Review comment:
       I suppose it is a shallow copy, but the doc says it is deep copy?

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {
+  auto node = make_object<MeasureInputNode>();
+  node->task = task;
+  node->state = state;
+  return MeasureInput(node);
+}
+
+BuildResult::BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+                         double time_cost) {
+  auto node = make_object<BuildResultNode>();
+  node->filename = std::move(filename);
+  node->args = std::move(args);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->time_cost = time_cost;
+  data_ = std::move(node);
+}
+
+MeasureResult::MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                             double timestamp) {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = std::move(costs);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  data_ = std::move(node);
+}
+
+MeasureResult MeasureResultNode::copy() const {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = costs;
+  node->error_no = error_no;
+  node->error_msg = error_msg;
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  return MeasureResult(node);
+}
+
+/********** LocalBuilder **********/
+LocalBuilder::LocalBuilder(int timeout, int n_parallel, const String& build_func) {
+  auto node = make_object<LocalBuilderNode>();
+  node->timeout = timeout;
+  node->n_parallel = n_parallel;
+  node->build_func = build_func;
+  data_ = std::move(node);
+}
+
+Array<BuildResult> LocalBuilderNode::Build(const Array<MeasureInput>& inputs, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_builder.build")) {
+    Array<BuildResult> results = (*f)(inputs, timeout, n_parallel, build_func, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_builder.build is not registered";
+  }
+  return Array<BuildResult>();
+}
+
+/********** LocalRunner **********/
+LocalRunner::LocalRunner(int timeout, int number, int repeat, int min_repeat_ms,
+                         double cooldown_interval) {
+  ObjectPtr<LocalRunnerNode> node = make_object<LocalRunnerNode>();
+  node->timeout = timeout;
+  node->number = number;
+  node->repeat = repeat;
+  node->min_repeat_ms = min_repeat_ms;
+  node->cooldown_interval = cooldown_interval;
+  data_ = std::move(node);
+}
+
+Array<MeasureResult> LocalRunnerNode::Run(const Array<MeasureInput>& inputs,
+                                          const Array<BuildResult>& build_results, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_runner.run")) {
+    Array<MeasureResult> results = (*f)(inputs, build_results, timeout, number, repeat,
+                                        min_repeat_ms, cooldown_interval, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_runner.run is not registered";
+  }
+  return Array<MeasureResult>();

Review comment:
       Same here

##########
File path: src/ansor/utils.h
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_ANSOR_UTILS_H_
+#define TVM_ANSOR_UTILS_H_
+
+#include <dmlc/common.h>
+#include <tvm/tir/expr.h>
+
+#include <algorithm>
+#include <deque>
+#include <exception>
+#include <future>
+#include <string>
+#include <thread>
+#include <tuple>
+#include <utility>
+#include <vector>
+
+namespace std {
+
+/*! \brief Hash function for std::pair */
+template <typename T1, typename T2>
+struct hash<std::pair<T1, T2>> {
+  std::size_t operator()(const std::pair<T1, T2>& k) const {
+    return ::dmlc::HashCombine(std::hash<T1>()(k.first), std::hash<T2>()(k.second));
+  }
+};
+
+/*! \brief Hash function for std::tuple */
+template <typename T1, typename T2, typename T3>
+struct hash<std::tuple<T1, T2, T3>> {
+  std::size_t operator()(const std::tuple<T1, T2, T3>& k) const {
+    return ::dmlc::HashCombine(
+        ::dmlc::HashCombine(std::hash<T1>()(std::get<0>(k)), std::hash<T2>()(std::get<1>(k))),
+        std::hash<T3>()(std::get<2>(k)));
+  }
+};
+
+}  // namespace std
+
+namespace tvm {
+namespace ansor {
+
+/********** Utilities for Array, std::string **********/
+/*! \brief Get the first appearance index of elements in an Array */
+template <typename T>
+inline void GetIndices(const Array<T>& array, const Array<T>& to_locate, Array<Integer>* indices) {
+  for (const auto& v : to_locate) {
+    auto it = std::find(array.begin(), array.end(), v);
+    if (it != array.end()) {
+      indices->push_back(it - array.begin());
+    } else {
+      LOG(FATAL) << "Cannot find the item";
+    }
+  }
+}
+
+/*! \brief Get the first appearance index of an element in an Array */
+template <typename T>
+inline int GetIndex(const Array<T>& array, const T& to_locate) {
+  for (size_t i = 0; i < array.size(); ++i) {
+    if (array[i] == to_locate) {
+      return i;
+    }
+  }
+  LOG(FATAL) << "Cannot find the item";
+  return -1;
+}
+
+/*! \brief Replace a sub-string to another sub-string in a string */
+inline void StrReplace(std::string* base, const std::string& from, const std::string& to) {
+  auto pos = base->find(from);
+  while (pos != std::string::npos) {
+    base->replace(pos, from.size(), to);
+    pos = base->find(from, pos + to.size());
+  }
+}
+
+/********** Utilities for TVM Containers / ByteArray **********/
+/*! \brief Compute mean of a FloatImm array */
+inline double FloatArrayMean(const Array<PrimExpr>& float_array) {
+  double sum = 0;
+  if (float_array.empty()) {
+    return 0.0;
+  }
+
+  for (const auto& x : float_array) {
+    auto floatimm = x.as<tir::FloatImmNode>();
+    CHECK(floatimm != nullptr);
+    sum += floatimm->value;
+  }
+  return sum / float_array.size();
+}
+
+/********** Other Utilities **********/
+/*! \brief Get an int value from an Expr */
+inline int64_t GetIntImm(const PrimExpr& expr) {
+  auto pint = expr.as<IntImmNode>();
+  CHECK(pint != nullptr);
+  return pint->value;
+}
+
+/*! \brief Compute the product of the lengths of axes */
+inline int64_t AxisLengthProd(const Array<tir::IterVar>& axes) {
+  int64_t ret = 1.0;
+  for (const auto& x : axes) {
+    if (const IntImmNode* imm = x->dom->extent.as<IntImmNode>()) {
+      ret *= imm->value;
+    } else {
+      return -1.0;
+    }
+  }
+  return ret;
+}
+
+/*!
+ * \brief Clean the name of an iterator to make it valid in python code.
+ * \param str The original name.
+ * \return The cleaned name.
+ */
+inline std::string CleanName(const std::string& str) {
+  std::string ret = str;
+  StrReplace(&ret, ".", "_");
+  StrReplace(&ret, "@", "_");
+  StrReplace(&ret, "outer", "o");
+  StrReplace(&ret, "inner", "i");
+  return ret;
+}
+
+/*! \brief An empty output stream */
+class NullStream : public std::ostream {
+ public:
+  NullStream() : std::ostream(nullptr) {}
+  NullStream(const NullStream&) : std::ostream(nullptr) {}
+  static NullStream& Global();
+};
+
+template <class T>
+NullStream& operator<<(NullStream& os, const T& value) {
+  return os;
+}
+
+/*! \brief Get std cout with verbose control */
+inline std::ostream& StdCout(int verbose) {
+  return verbose == 1 ? std::cout : NullStream::Global();

Review comment:
       Do we really want to use `std::cout`? Maybe `LOG(INFO)` or `LOG(VERBOSE)`?

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {
+  auto node = make_object<MeasureInputNode>();
+  node->task = task;
+  node->state = state;
+  return MeasureInput(node);
+}
+
+BuildResult::BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+                         double time_cost) {
+  auto node = make_object<BuildResultNode>();
+  node->filename = std::move(filename);
+  node->args = std::move(args);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->time_cost = time_cost;
+  data_ = std::move(node);
+}
+
+MeasureResult::MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                             double timestamp) {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = std::move(costs);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  data_ = std::move(node);
+}
+
+MeasureResult MeasureResultNode::copy() const {

Review comment:
       It is shallow copy too.

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);

Review comment:
       Is there specific rules why some of them are registered using `OBJECT_TYPE` but the others are `NODE_TYPE`? IIRC `NODE_TYPE` helps register creator into the vtable, so my understand is that if we don't customize our own creator, just use `NODE_TYPE` instead.

##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * The flow of data structures is MeasureInput -> BuildeResult -> MeasureResult.
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;

Review comment:
       Use `ShallowCopy`/`DeepCopy` to deliver clearer information.

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {
+  auto node = make_object<MeasureInputNode>();
+  node->task = task;
+  node->state = state;
+  return MeasureInput(node);
+}
+
+BuildResult::BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+                         double time_cost) {
+  auto node = make_object<BuildResultNode>();
+  node->filename = std::move(filename);
+  node->args = std::move(args);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->time_cost = time_cost;
+  data_ = std::move(node);
+}
+
+MeasureResult::MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                             double timestamp) {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = std::move(costs);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  data_ = std::move(node);
+}
+
+MeasureResult MeasureResultNode::copy() const {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = costs;
+  node->error_no = error_no;
+  node->error_msg = error_msg;
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  return MeasureResult(node);
+}
+
+/********** LocalBuilder **********/
+LocalBuilder::LocalBuilder(int timeout, int n_parallel, const String& build_func) {
+  auto node = make_object<LocalBuilderNode>();
+  node->timeout = timeout;
+  node->n_parallel = n_parallel;
+  node->build_func = build_func;
+  data_ = std::move(node);
+}
+
+Array<BuildResult> LocalBuilderNode::Build(const Array<MeasureInput>& inputs, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_builder.build")) {
+    Array<BuildResult> results = (*f)(inputs, timeout, n_parallel, build_func, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_builder.build is not registered";
+  }
+  return Array<BuildResult>();

Review comment:
       Don't have to return anything because it is fatal anyways.
   
   ```suggestion
     }
     LOG(FATAL) << "ansor.local_builder.build is not registered";
     throw;
   ```

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {
+  auto node = make_object<MeasureInputNode>();
+  node->task = task;
+  node->state = state;
+  return MeasureInput(node);
+}
+
+BuildResult::BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+                         double time_cost) {
+  auto node = make_object<BuildResultNode>();
+  node->filename = std::move(filename);
+  node->args = std::move(args);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->time_cost = time_cost;
+  data_ = std::move(node);
+}
+
+MeasureResult::MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                             double timestamp) {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = std::move(costs);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  data_ = std::move(node);
+}
+
+MeasureResult MeasureResultNode::copy() const {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = costs;
+  node->error_no = error_no;
+  node->error_msg = error_msg;
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  return MeasureResult(node);
+}
+
+/********** LocalBuilder **********/
+LocalBuilder::LocalBuilder(int timeout, int n_parallel, const String& build_func) {
+  auto node = make_object<LocalBuilderNode>();
+  node->timeout = timeout;
+  node->n_parallel = n_parallel;
+  node->build_func = build_func;
+  data_ = std::move(node);
+}
+
+Array<BuildResult> LocalBuilderNode::Build(const Array<MeasureInput>& inputs, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_builder.build")) {
+    Array<BuildResult> results = (*f)(inputs, timeout, n_parallel, build_func, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_builder.build is not registered";
+  }
+  return Array<BuildResult>();
+}
+
+/********** LocalRunner **********/
+LocalRunner::LocalRunner(int timeout, int number, int repeat, int min_repeat_ms,
+                         double cooldown_interval) {
+  ObjectPtr<LocalRunnerNode> node = make_object<LocalRunnerNode>();
+  node->timeout = timeout;
+  node->number = number;
+  node->repeat = repeat;
+  node->min_repeat_ms = min_repeat_ms;
+  node->cooldown_interval = cooldown_interval;
+  data_ = std::move(node);
+}
+
+Array<MeasureResult> LocalRunnerNode::Run(const Array<MeasureInput>& inputs,
+                                          const Array<BuildResult>& build_results, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_runner.run")) {
+    Array<MeasureResult> results = (*f)(inputs, build_results, timeout, number, repeat,
+                                        min_repeat_ms, cooldown_interval, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_runner.run is not registered";
+  }
+  return Array<MeasureResult>();
+}
+
+/********** ProgramMeasurer **********/
+ProgramMeasurer::ProgramMeasurer(ProgramBuilder builder, ProgramRunner runner,
+                                 Array<MeasureCallback> callbacks, int verbose,
+                                 int max_continous_error) {
+  auto node = make_object<ProgramMeasurerNode>();
+  node->builder = std::move(builder);
+  node->runner = std::move(runner);
+  node->callbacks = std::move(callbacks);
+  node->verbose = verbose;
+  node->max_continous_error = max_continous_error < 0
+                                  ? ProgramMeasurerNode::DEFAULT_MAX_CONTINOUS_ERROR
+                                  : max_continous_error;
+  data_ = std::move(node);
+}
+
+void ProgramMeasurerNode::Reset() {
+  ct = error_ct = 0;
+  best_flops.clear();
+  best_ct.clear();
+  best_state.clear();
+}
+
+void ProgramMeasurerNode::Measure(const SearchTask& task, const SearchPolicy& policy,
+                                  const Array<MeasureInput>& inputs, Array<MeasureResult>* results,
+                                  int batch_size) {
+  results->clear();
+  results->reserve(inputs.size());
+
+  if (batch_size == -1) {
+    // set default batch size
+    batch_size = builder->n_parallel * 2;
+  }
+
+  StdCout(verbose) << "Get " << inputs.size() << " programs for measure. (This may take a while)"
+                   << std::endl;
+
+  for (size_t i = 0; i < inputs.size(); i += batch_size) {
+    Array<MeasureInput> input_batch(inputs.begin() + i,
+                                    inputs.begin() + std::min(i + batch_size, inputs.size()));
+    Array<MeasureResult> result_batch;
+
+    // build and run
+    SilentMeasure(task, input_batch, &result_batch);
+
+    // update current best state according to the new measure result
+    for (size_t j = 0; j < input_batch.size(); ++j) {
+      double flops;
+      if (result_batch[j]->error_no == 0) {
+        flops = task->compute_dag->flop_ct / FloatArrayMean(result_batch[j]->costs);
+        error_ct = 0;
+      } else {
+        flops = 0.0;
+        error_ct++;
+      }
+
+      const String& workload_key = input_batch[j]->task->workload_key;
+      if (flops > best_flops[workload_key]) {
+        best_flops[workload_key] = flops;
+        best_state[workload_key] = input_batch[j]->state;
+        best_ct[workload_key] = ct;
+      }
+
+      ct++;
+      StdCout(verbose) << std::fixed << std::setprecision(2) << Chars('=', 50) << "\n"
+                       << "No: " << ct << "\tGFLOPS: " << flops / 1e9 << " / "
+                       << best_flops[workload_key] / 1e9 << "\tresults: " << result_batch[j] << "\n"
+                       << Chars('=', 50) << "\n"
+                       << input_batch[j]->state << "\n";
+    }
+
+    // Call callback functions
+    for (const auto& callback : callbacks) {
+      callback->Callback(policy, input_batch, result_batch);
+    }
+
+    // Store result batch
+    for (auto& res : result_batch) {
+      results->push_back(res);
+    }
+
+    if (error_ct > max_continous_error) {
+      LOG(FATAL) << "Too many errors happened during tuning";
+    }
+  }
+}
+
+void ProgramMeasurerNode::SilentMeasure(const SearchTask& task, const Array<MeasureInput>& inputs,
+                                        Array<MeasureResult>* results) {
+  // Close the thread pool to avoid the conflits with python environment
+  ThreadPool::Global().Abort();
+
+  results->clear();
+  results->reserve(inputs.size());
+  Array<MeasureInput> input_batch(inputs.begin(), inputs.end());

Review comment:
       Why we need a copy here?

##########
File path: src/ansor/measure.cc
##########
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.cc
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ */
+
+#include "measure.h"
+
+#include <tvm/runtime/registry.h>
+
+#include <algorithm>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(MeasureInputNode);
+TVM_REGISTER_NODE_TYPE(BuildResultNode);
+TVM_REGISTER_NODE_TYPE(MeasureResultNode);
+TVM_REGISTER_OBJECT_TYPE(MeasureCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramRunnerNode);
+TVM_REGISTER_OBJECT_TYPE(ProgramBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalBuilderNode);
+TVM_REGISTER_OBJECT_TYPE(LocalRunnerNode);
+
+static const char* ErrorNoToStr[] = {
+    "NoError",
+    "InstantiationError",
+    "CompileHostError",
+    "CompileDeviceError",
+    "RuntimeDeviceError",
+    "WrongAnswerError",
+    "BuildTimeoutError",
+    "RunTimeoutError",
+    "UnknownError",
+};
+
+/********** Measure input and result **********/
+MeasureInput::MeasureInput(SearchTask task, State state) {
+  auto node = make_object<MeasureInputNode>();
+  node->task = std::move(task);
+  node->state = std::move(state);
+  data_ = std::move(node);
+}
+
+MeasureInput MeasureInputNode::copy() const {
+  auto node = make_object<MeasureInputNode>();
+  node->task = task;
+  node->state = state;
+  return MeasureInput(node);
+}
+
+BuildResult::BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+                         double time_cost) {
+  auto node = make_object<BuildResultNode>();
+  node->filename = std::move(filename);
+  node->args = std::move(args);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->time_cost = time_cost;
+  data_ = std::move(node);
+}
+
+MeasureResult::MeasureResult(Array<PrimExpr> costs, int error_no, String error_msg, double all_cost,
+                             double timestamp) {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = std::move(costs);
+  node->error_no = error_no;
+  node->error_msg = std::move(error_msg);
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  data_ = std::move(node);
+}
+
+MeasureResult MeasureResultNode::copy() const {
+  auto node = make_object<MeasureResultNode>();
+  node->costs = costs;
+  node->error_no = error_no;
+  node->error_msg = error_msg;
+  node->all_cost = all_cost;
+  node->timestamp = timestamp;
+  return MeasureResult(node);
+}
+
+/********** LocalBuilder **********/
+LocalBuilder::LocalBuilder(int timeout, int n_parallel, const String& build_func) {
+  auto node = make_object<LocalBuilderNode>();
+  node->timeout = timeout;
+  node->n_parallel = n_parallel;
+  node->build_func = build_func;
+  data_ = std::move(node);
+}
+
+Array<BuildResult> LocalBuilderNode::Build(const Array<MeasureInput>& inputs, int verbose) {
+  if (const auto* f = runtime::Registry::Get("ansor.local_builder.build")) {
+    Array<BuildResult> results = (*f)(inputs, timeout, n_parallel, build_func, verbose);
+    return results;
+  } else {
+    LOG(FATAL) << "ansor.local_builder.build is not registered";

Review comment:
       I think we should document where `ansor.local_builder.build` is defined, and hint some potential reasons why it is not found (e.g. python package not fully loaded)




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    target : tvm.target.Target
+        The target device of this search task.
+    target_host : tvm.target.Target
+        The target host device of this search task.
+    hardware_params : HardwareParams
+        Hardware parameters used in this search task.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ The options for tuning
+
+    Parameters
+    ----------
+    n_trials: int
+      Number of total measurement trials
+    early_stopping: int
+      Stops early the tuning if no improvement after n measurements
+    num_measure_per_round: int
+      The number of programs to be measured at each iteration
+    verbose: int
+      Verbosity level. 0 means silent.
+    builder: Builder
+      Builder which builds the program
+    runner: Runner
+      Runner which runs the program and measure time costs
+    measure_callbacks: List[MeasureCallback]
+      Callback functions called after each measure
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: List[SearchCallback]
+      Callback functions called before the search process
+      Candidates:
+        - ansor.PreloadMeasuredStates(will be added later)
+        - ansor.PreloadCustomSketchRule(will be added later)
+    """
+    def __init__(self, n_trials=0, early_stopping=-1, num_measure_per_round=64,
+                 verbose=1, builder='local', runner='local', measure_callbacks=None,
+                 pre_search_callbacks=None):
+        if isinstance(builder, str):
+            if builder == 'local':
+                builder = LocalBuilder()
+            else:
+                raise ValueError("Invalid builder: " + builder)
+
+        if isinstance(runner, str):
+            if runner == 'local':
+                runner = LocalRunner()
+            else:
+                raise ValueError("Invalid builder: " + runner)
+
+        if measure_callbacks is None:
+            measure_callbacks = []
+
+        if pre_search_callbacks is None:
+            pre_search_callbacks = []
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.TuneOption, n_trials, early_stopping, num_measure_per_round,
+            verbose, builder, runner, measure_callbacks, pre_search_callbacks)
+
+
+def auto_schedule(workload, target=None,
+                  target_host=None, search_policy='default',
+                  hardware_params=None, tune_option=None):
+    """ Do auto scheduling for a computation declaration.
+
+    The workload parameter can be a `string` as workload_key, or directly
+    passing a `SearchTask` as input.
+
+    Parameters
+    ----------
+    workload : Union[SearchTask, str]
+        The target search task or workload key.
+    target : Target
+        The target device of this schedule search.
+    target_host : Target = None
+        The target host device of this schedule search.
+    search_policy : Union[SearchPolicy, str]
+        The search policy to be used for schedule search.
+    hardware_params : HardwareParams
+        The hardware parameters of this schedule search.
+    tune_option : TuneOption
+        Tuning and measurement options.
+
+    Returns
+    -------
+        A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+    """
+    if isinstance(search_policy, str):
+        if search_policy == 'default':
+            search_policy = EmptyPolicy()
+        else:
+            raise ValueError("Invalid search policy: " + search_policy)
+
+    if tune_option is None:
+        tune_option = TuneOption(n_trials=0)

Review comment:
       Per offline discussion, we should rename `n_trial` to a more straightforward name like `n_measure_trial` to indicate that this is for measurement. We should also have another `n_trial` (or any better name) for the cost model. With those two in the TuneOption, I think `n_measure_trial=0` is easy to understand.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considered to be merged with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the corresponding compute declaration.

Review comment:
       In my opinion, this seems more likly the explainations to the arg classes. It may be better for users to get that from their own docs.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,466 @@
+# 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.
+
+"""
+Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling.
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# We use fork and a global variable to copy arguments between processings.
+# This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The SearchTask of this measure.
+    state : State
+        The State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbose : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbose : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        The number of times to run the generated code for taking average.
+        We call these runs as one `repeat` of measurement.
+    repeat : int = 1
+        The number of times to repeat the measurement.
+        In total, the generated code will be run (1 + number x repeat) times,
+        where the first "1" is warm up and will be discarded.
+        The returned result contains `repeat` costs,
+        each of which is an average of `number` costs.
+    min_repeat_ms : int = 0
+        The minimum duration of one `repeat` in milliseconds.
+        By default, one `repeat` contains `number` runs. If this parameter is set,
+        the parameters `number` will be dynamically adjusted to meet the
+        minimum duration requirement of one `repeat`.
+        i.e., When the run time of one `repeat` falls below this time, the `number` parameter
+        will be automatically increased.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \

Review comment:
       Some messages are too long, making the log output unreadable




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int

Review comment:
       This should be part of the Target later https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844 cc @comaniac 




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.cc
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.cc
+ * \brief The user interface of the Ansor auto-scheduler.
+ */
+
+#include "auto_schedule.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(TuningOptionsNode);
+
+TuningOptions::TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round,
+                             int verbose, ProgramBuilder builder, ProgramRunner runner,
+                             Array<MeasureCallback> measure_callbacks,
+                             Array<SearchCallback> pre_search_callbacks) {
+  auto node = make_object<TuningOptionsNode>();

Review comment:
       Good point! ... Seems all constructors in TVM code base are in this style, I just follow that.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.

Review comment:
       Remove all "target" before "MeasureInputs", "MeasureResults" or "compute declaration" in all files




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class of search policies, including the abstract definition of search policy and
+ * other supporting data structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+ * initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+ * uses cost model based evolutionary search to select schedules with the best performance.
+ *
+ * Candidate schedules are measured against the specific hardware target.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover most use cases. Meanwhile, a custom rule
+ * mechanism will be provided to enable user-defined template search to serve the same functionality
+ * as the current AutoTVM template.
+ *
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search process and returns the best schedule get.
+ * 2. Information about the compute declaration of ops/subgraphs can be acquired from SearchTask.
+ * This structure also contains some information about the target device. (e.g. knowing the weight
+ * of the device vector unit, we can limit the max vectorize size during schedule generating)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */

Review comment:
       ```suggestion
   This guide is for advanced uses who have special requirements.
    * 1. The only function that must be implemented is Search(), which takes a task as input and returns the best states found.
    * 2. Information about the compute declaration of ops/subgraphs can be acquired from SearchTask.
    * This structure also contains some information about the target device. (e.g. knowing the width
    * of the device vector unit, we can limit the max vectorize size during schedule search)
    * 3. SearchCallback provides more flexibility to do extra affairs before/after the search process.
    * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states got
    * during the search process.
    */
   ```

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api
+
+  /*! \brief The max shared memory per block. */
+  int max_shared_memory_per_block{INT32_MAX};
+  /*! \brief The max register memory per block. */
+  int max_registers_per_block{INT32_MAX};
+  /*! \brief The max threads per block. */
+  int max_threads_per_block{INT32_MAX};
+  /*! \brief The max vthread extent. */
+  int max_vthread_extent{INT32_MAX};
+  /*! \brief The thread numbers of a warp. */
+  int warp_size{INT32_MAX};
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_cores", &num_cores);
+    v->Visit("vector_unit_bytes", &vector_unit_bytes);
+    v->Visit("cache_line_bytes", &cache_line_bytes);
+    v->Visit("max_shared_memory_per_block", &max_shared_memory_per_block);
+    v->Visit("max_registers_per_block", &max_registers_per_block);
+    v->Visit("max_threads_per_block", &max_threads_per_block);
+    v->Visit("max_vthread_extent", &max_vthread_extent);
+    v->Visit("warp_size", &warp_size);
+  }
+
+  /*!
+   * \brief Get the default hardware params.
+   * \param target A `tvm.target`.
+   * \param target_host A `tvm.target` for host device.
+   * \return A HardwareParams object.
+   */
+  static HardwareParams GetDefaultHardwareParams(const Target& target, const Target& target_host);
+
+  static constexpr const char* _type_key = "ansor.HardwareParams";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HardwareParamsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to HardwareParamsNode.
+ * \sa HardwareParamsNode
+ */
+class HardwareParams : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param num_cores The number of cores.
+   * \param vector_unit_bytes The width of vector units in bytes.
+   * \param cache_line_bytes The size of cache line in bytes.
+   */
+  HardwareParams(int num_cores, int vector_unit_bytes, int cache_line_bytes);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(HardwareParams, ObjectRef, HardwareParamsNode);
+  TVM_DEFINE_OBJECT_REF_COW_METHOD(HardwareParamsNode);
+};
+
+/*!
+ * \brief The computation information and hardware parameters for a specific schedule search task.
+ */
+class SearchTaskNode : public Object {
+ public:
+  /*! \brief The ComputeDAG for target compute declaration. */

Review comment:
       replace all "target compute declaration" with "input compute declaration" in this file

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class of search policies, including the abstract definition of search policy and
+ * other supporting data structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+ * initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+ * uses cost model based evolutionary search to select schedules with the best performance.
+ *
+ * Candidate schedules are measured against the specific hardware target.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover most use cases. Meanwhile, a custom rule
+ * mechanism will be provided to enable user-defined template search to serve the same functionality
+ * as the current AutoTVM template.
+ *
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search process and returns the best schedule get.
+ * 2. Information about the compute declaration of ops/subgraphs can be acquired from SearchTask.
+ * This structure also contains some information about the target device. (e.g. knowing the weight
+ * of the device vector unit, we can limit the max vectorize size during schedule generating)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to a SearchPolicyNode.
+   */
+  virtual void Callback(SearchPolicyNode* policy) = 0;
+
+  static constexpr const char* _type_key = "ansor.SearchCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchCallbackNode.
+ * \sa SearchCallbackNode
+ */
+class SearchCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(SearchCallback, ObjectRef, SearchCallbackNode);
+};
+
+/*!
+ * \brief The base class of search policies.
+ */
+class SearchPolicyNode : public Object {
+ public:
+  /*! \brief The current search task. */
+  SearchTask cur_task;
+  /*!
+   * \brief Verbose level to control the screen output during schedule search.
+   * 0 for silent, 1 to output information.
+   */
+  int verbose;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("cur_task", &cur_task);
+    v->Visit("verbose", &verbose);
+  }
+
+  /*!
+   * \brief Do schedule search for a task. Takes the SearchTask as input and returns the best state
+   * get during the search process.
+   * \param task The target search task.

Review comment:
       replace `target search task` with `search task` in all files

##########
File path: src/ansor/search_policy/empty_policy.h
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/empty_policy.h
+ * \brief This is an brief example of search policy.

Review comment:
       ```suggestion
    * \brief A brief example of the search policy which always returns the initial naive schedule (state)
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+import pickle
+import json
+
+import tvm._ffi
+from .utils import serialize_args, deserialize_args
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload(func):
+    """ Register a workload by generation function.
+
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.

Review comment:
       Remove all "target" in this file. Replace it with "task" if you really want something before the noun.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.

Review comment:
       Remove all "target"  before "SearchTask", "search task", "compute declaration". Replace it with "input" if you really want something before the noun.

##########
File path: src/ansor/search_task.h
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_task.h
+ * \brief Meta information and hardware parameters for a search task.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_TASK_H_
+#define TVM_ANSOR_SEARCH_TASK_H_
+
+#include <tvm/target/target.h>
+
+#include "compute_dag.h"
+
+namespace tvm {
+namespace ansor {
+
+class HardwareParams;
+
+/*! \brief The parameters of target hardware used to guide the search process of SearchPolicy. */
+class HardwareParamsNode : public Object {
+ public:
+  /*! \brief The number of cores. */
+  int num_cores;
+  /*! \brief The width of vector units in bytes. */
+  int vector_unit_bytes;
+  /*! \brief The size of cache line in bytes. */
+  int cache_line_bytes;
+
+  // Some GPU related limitations
+  // Get from TVM device api

Review comment:
       ```suggestion
     // GPU related parameters got from device query API
   ```

##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult

Review comment:
       ```suggestion
    * The flow of data structures is MeasureInput -> BuildeResult -> MeasureResult.
   ```

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to write log to.
+    inputs: List[MeasureInputs]
+        The target MeasureInputs to be written.

Review comment:
       Remove all "target" before "MeasureInputs", "MeasureResults" or "compute declaration" in this file

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The target range of this iterator. */

Review comment:
       What's the meaning of "target range"?
   Remove all "target" in this file.

##########
File path: src/ansor/search_policy/search_policy.h
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.h
+ * \brief The base class of search policies, including the abstract definition of search policy and
+ * other supporting data structures.
+ *
+ * The basic schedule search process for Ansor is design to be:
+ * `Program sampling` -> `Performance Tuning`.
+ *
+ * In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+ * initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+ * uses cost model based evolutionary search to select schedules with the best performance.
+ *
+ * Candidate schedules are measured against the specific hardware target.
+ *
+ * \note Adding a new search policy.
+ * In design, there's no need for users to implement their own search policy, our formal search
+ * policy(will be brought later) should be enough to cover most use cases. Meanwhile, a custom rule
+ * mechanism will be provided to enable user-defined template search to serve the same functionality
+ * as the current AutoTVM template.
+ *
+ * This guide is to help understand it better and incase some advanced users have special
+ * requirements.
+ * 1. The only funcion that must be implemented is Search(), the design principe for it is to be
+ * the entry of starting a schedule search process and returns the best schedule get.
+ * 2. Information about the compute declaration of ops/subgraphs can be acquired from SearchTask.
+ * This structure also contains some information about the target device. (e.g. knowing the weight
+ * of the device vector unit, we can limit the max vectorize size during schedule generating)
+ * 3. SearchCallback provides more flexibility to do extra affairs during the search process.
+ * 4. ProgramMeasurer provides a simple but useful api to help check the performance of states get
+ * during the search process.
+ */
+
+#ifndef TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+#define TVM_ANSOR_SEARCH_POLICY_SEARCH_POLICY_H_
+
+#include <tvm/node/node.h>
+
+#include <unordered_set>
+#include <vector>
+
+#include "../search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class ProgramMeasurer;
+class SearchPolicyNode;
+
+/*!
+ * \brief Callback function to be called by the search process.
+ * This interface allows to do extra initializations before schedule search or extra
+ * check during/after the schedule search.
+ */
+class SearchCallbackNode : public Object {
+ public:
+  /*!
+   * \brief Run the registered callback function.
+   * \param policy A pointer to a SearchPolicyNode.
+   */
+  virtual void Callback(SearchPolicyNode* policy) = 0;
+
+  static constexpr const char* _type_key = "ansor.SearchCallback";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchCallbackNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SearchCallbackNode.
+ * \sa SearchCallbackNode
+ */
+class SearchCallback : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(SearchCallback, ObjectRef, SearchCallbackNode);
+};
+
+/*!
+ * \brief The base class of search policies.
+ */
+class SearchPolicyNode : public Object {
+ public:
+  /*! \brief The current search task. */
+  SearchTask cur_task;
+  /*!
+   * \brief Verbose level to control the screen output during schedule search.
+   * 0 for silent, 1 to output information.
+   */
+  int verbose;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("cur_task", &cur_task);
+    v->Visit("verbose", &verbose);
+  }
+
+  /*!
+   * \brief Do schedule search for a task. Takes the SearchTask as input and returns the best state
+   * get during the search process.
+   * \param task The target search task.
+   * \param num_measure_trials Total schedules to be tried during this search.
+   * \param early_stopping Early stop if no better schedule is found.
+   * \param num_measures_per_round Max measure batch in one search round.
+   * \param verbose Verbose level. 0 for silent, 1 to output information during schedule search.
+   * \param measurer A ProgramMeasurer which packs ProgramBuilder & ProgramRunner inside.
+   * \param pre_search_callbacks SearchCallback to be called before schedule search.
+   * \return The best state get.
+   */
+  virtual State Search(SearchTask task, int num_measure_trials, int early_stopping,
+                       int num_measures_per_round, int verbose, ProgramMeasurer measurer,
+                       Array<SearchCallback> pre_search_callbacks) = 0;
+
+  /*!
+   * \brief Call SearchCallback with the current SearchPolicyNode
+   * \param callbacks SearchCallback to be called.
+   */
+  void RunCallbacks(const Array<SearchCallback>& callbacks);
+
+  static constexpr const char* _type_key = "ansor.SearchPolicy";
+  TVM_DECLARE_BASE_OBJECT_INFO(SearchPolicyNode, Object);
+
+ protected:
+  /*!
+   * \brief The set of already measured states.
+   * We store the string format for redundancy check.
+   */
+  std::unordered_set<String> measured_states_set_;

Review comment:
       For this internal `std::unordered_set`, I do not think we need to use tvm's String




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""

Review comment:
       ```suggestion
   The definition of the "state" in search.
   
   Each LoopState corresponds to a schedule for its ComputeDAG.
   A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
   construct the loop structure.
   The loop structure keeps a preview of how the schedule will finally look like after lowering the
   current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
   During the schedule search process, the loop structure can provide search policy with necessary
   information on how to manipulate the current state.
   The transform history is a sequence of `TransformStep` which will finally be mapped to TVM schedule
   primitives. The steps can also be used for the serialization of a state.
   
   The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
   We don't use the existing TVM IR but to extend a new structure on it is because:
   1. We want fast incremental change to the loop structures. The search policy needs to get the immediate
   loop structures update rather than after TVM lowering;
   2. We want serializable transform history for replay, backtracking, and mutation;
   3. We may create some macro schedule primitives that represent the combination of several
   TVM schedule primitives.
   
   When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
   Since we share a lot of common objects during search, the transformation is implemented in
   copy on write style. All objects are immutable, which is similar to TVM IR.
   """
   ```
   
   
   
   Also, propagate the changes to c++ files.
   




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.

Review comment:
       This is just for debugging and pretty printing




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+ * construct it.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...). During the schedule search process, the loop structure can provide search policy with
+ * necessary information on how to perform further operations with the current state.
+ * The transform history is a sequence of TransformStep which will finally be mapped to schedule
+ * primitives. The steps can also be used for serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures, search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several TVM
+ * schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in copy
+ * on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */

Review comment:
       Actually these annotations here are not used in this PR, since this is going to bring a minimum system.
   Btw, the reason we did not add thread/block z is because our search policy only used thread/block x/y. They can be added easily if needed.




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       wondering why it is double btw? potential overflow?




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       Thanks @jcf94 , let me try to elaborate further. To simplify the abstraction, we should:
   
   - Add src/support/parallel_for.h
      - Move the threadpool as a detail of parallel_for.cc, remove thread_pool from utils.h
      - It is unclear whether threadpool is needed to implement parallel for, it is very possible that we can just launch n std::thread(because std::thread is quite lightweight in c++)
   - Use parallel_for for all necessary usecases of threadpool.
   
   Also consider remove the stride argument, or make it optional since stride is not used.
   
   




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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


   Thank all the reviewers ! Most comments from the reviewers are addressed.
   
   We summarize the unresolved comments here. They will be addressed by follow-up PRs.
   - The design of `python/tvm/auto_schedule/auto_scheduler.py::auto_schedule`. This is the most important user interface to use the auto-scheduler. We will send an RFC to discuss this.
   - Move `src/auto_schedule/utils.h::ThreadPool` to `src/support/parallle_for.h` (create a new file)
   - Consolidate the usage of `python/tvm/auto_schedule/utils.py::get_const_int`, `python/tvm/auto_schedule/utils.py::get_const_tuple` in the whole TVM code base.
   
   @jroesch @jwfromm @yangjunpro @MarisaKirisame @junrushao1994  Please take another look and approve.


----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]

Review comment:
       This confused me at first....
   The workload_key is a string of `'["matmul_ansor_test", 128, 128, 128]'`,
   while the workload is a list `["matmul_ansor_test", 128, 128, 128]`.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the compute declaration,
+    a list of all operations in the DAG as well as static analysis results for the DAG (e.g. the
+    total float operation count, consumer/producer relations of each operation stage, whether an
+    operation stage should be tiled/compute inlined ...). These analyses can help the search policy
+    to make decisions during search process.
+    ComputeDAG is also responsible for the interaction between Ansor `LoopState` and TVM schedule
+    (e.g. applying the `LoopState` transform steps to TVM schedule, providing `LoopState` with extra
+    information got from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps from a State to get a TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The state from which we get transform steps.
+
+        Returns
+        -------
+            A `te.schedule` and the a list of `te.Tensor` to be used in `tvm.lower` or `tvm.build`.
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        This can be used for debugging or to apply the schedule on a former TVM version without
+        Ansor support.

Review comment:
       ```suggestion
           This is used to print transformation steps for debugging.
           Use `apply_steps_from_state` if you want to get a schedule for code generation.
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ Base class for measurement callback function. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.Builder")
+class Builder(Object):
+    """ Base class of Builder. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.BuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.Runner")
+class Runner(Object):
+    """ Base class of Runner """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.RunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(Builder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(Runner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):

Review comment:
       You can see the docstring below.
   We use fork and a global variable to copy arguments between processings with 
   This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
   




----------------------------------------------------------------
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] MarisaKirisame commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {

Review comment:
       auto_scheduler




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/auto_schedule/__init__.py
##########
@@ -0,0 +1,34 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       The namespace is auto_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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1

Review comment:
       There might be many verbosity levels so we should use int.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.

Review comment:
       We have the `apply_steps_from_state` the get the final schedule. This is used for debugging at the beginning, or to apply the schedule on a former TVM version without Ansor support.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,268 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).
+The format of the string is `[func_name, [args...]]`.
+The dag should be the return value of this `func_name(*args)`.
+
+Rationale: The workload is actually a compute dag defined by tvm dsl. But serializing compute dags
+and matching them efficiently is not easy. Therefore, we use the above string to encode a compute
+dag.
+These strings are efficient for serialization/matching and wont' be too long.
+When we need the dag, we decode the string and call the function, which will return the dag.
+"""
+
+from typing import Hashable
+import pickle
+import json
+import hashlib
+
+import tvm._ffi
+from ..te import Tensor, PlaceholderOp, ComputeOp, placeholder
+from .utils import get_const_tuple
+from .compute_dag import ComputeDAG
+
+WORKLOAD_FUNC_REGISTRY = {}
+
+
+def register_workload_func(func):
+    """Register a workload generation function
+    The input function should take hashable and jsonable arguments
+    (int, float, tuple of int, tvm.tensor.Tensor, ...) and return a list of tvm.tensor.Tensor.
+
+    Examples
+    --------
+    @register_workload_func
+    def matmul(N, M, K):
+        A = te.placeholder((N, K), name='A')
+        B = te.placeholder((K, M), name='B')
+        k = te.reduce_axis((0, K), name='k')
+        C = te.compute((N, M), lambda i, j: tvm.sum(A[i][k] * B[k][j], axis=[k]), name='C')
+        return [A, B, C]
+    """
+    func_name = func.__name__
+    if func_name in WORKLOAD_FUNC_REGISTRY:
+        raise RuntimeError('%s has been registered already' % func_name)
+    WORKLOAD_FUNC_REGISTRY[func_name] = func
+    return func
+
+
+def compute_dag_hash(dag):
+    """ Get hash value for a ComputeDAG.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The target ComputeDAG.
+
+    Returns
+    -------
+    hash_value : Str
+        The hash value of this ComputeDAG in hex digest.
+    """
+    # todo: implement this more carefully and move this to c++ as a member function of ComputeDAG
+    str_key = ''
+    for op in dag.ops:
+        t = op.output(0)
+        if isinstance(op, PlaceholderOp):
+            str_key += 'placeholder,'
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        elif isinstance(op, ComputeOp):
+            str_key += str(t.op.body) + ','
+            str_key += str(get_const_tuple(t.shape)) + ','
+            str_key += t.dtype + ';'
+        else:
+            raise ValueError("Invalid op: " + op)
+
+    str_key = str_key.encode(encoding='utf-8')
+    return hashlib.md5(str_key).hexdigest()
+
+
+def register_workload_bufs(bufs):
+    """ Directly register buffers of a workload and return the workload_key.
+
+    The buffers can be looked up with workload_key_to_tensors by the workload_key.
+
+    Parameters
+    ----------
+    bufs : List[Tensor]
+        A list of Tensors for the target compute declaration.
+
+    Returns
+    -------
+    workload_key : Str
+        A workload key mapping to the registered compute declaration.
+    """
+    dag = ComputeDAG(bufs)
+    key = compute_dag_hash(dag)
+    WORKLOAD_FUNC_REGISTRY[key] = bufs
+    return json.dumps((key,))
+
+
+def list_to_tuple(x):
+    """Convert a list to a tuple recursively"""
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+@tvm._ffi.register_func("ansor.workload_key_to_tensors")
+def workload_key_to_tensors(workload_key):
+    """ Decode a workload key to the input/output tensors.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    tensors : List[Tensor]
+        The registered compute declaration Tensors.
+    """
+    workload = json.loads(workload_key)
+    name = workload[0]
+    lookup = WORKLOAD_FUNC_REGISTRY[name]
+
+    if callable(lookup):
+        args = deserialize_args(workload[1:])
+        return lookup(*args)
+    return lookup
+
+
+@ tvm._ffi.register_func("ansor.workload_key_to_dag")
+def workload_key_to_dag(workload_key):
+    """ Decode a workload key to a compute dag.
+
+    Parameters
+    ----------
+    workload_key : Str
+        The target workload key.
+
+    Returns
+    -------
+    dag : ComputeDAG
+        ComputeDAG to the registered compute declaration.
+    """
+    tensors = workload_key_to_tensors(workload_key)
+    return ComputeDAG(tensors)
+
+
+def make_workload_key_func(func, args):
+    """ make a workload key from function and arguments.
+
+    Parameters
+    ----------
+    func : Function
+        The target function that returns the compute declaration Tensors.
+    args : Args
+        The args of the target function.
+
+    Returns
+    -------
+    workload_key : Str
+        The workload key of the target function.
+    """
+    args = serialize_args(args)
+
+    if callable(func):
+        func_name = func.__name__
+    elif isinstance(func, str):
+        func_name = func
+    else:
+        raise ValueError("Invalid function: " + str(func))
+
+    assert func_name in WORKLOAD_FUNC_REGISTRY, \

Review comment:
       I didn't quite get it. If we register a workload using `register_workload_by_func`, then we will trigger this assertion with the same function in another position (because it has been registered), right?




----------------------------------------------------------------
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] tqchen commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.h
##########
@@ -184,22 +184,36 @@ inline void PrintTitle(const std::string& title, int verbose) {
 }
 
 /*!
- * \brief A simple thread pool.
+ * \brief A simple thread pool to perform parallel for.
  * TODO(merrymercy): Move this to `src/support/parallel_for`
  */
-class ThreadPool {
+class ParallelFor {

Review comment:
       @jcf94 Sorry I wasn't meant to say that we should rename ThreadPool to ParallelFor, instead we should hide the use of threadpool behind a parallel_for API, in similar style to https://docs.microsoft.com/en-us/cpp/parallel/concrt/parallel-algorithms?view=vs-2019#parallel_for




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function

Review comment:
       This is not for global configuration.
   This is a temporary variable for copying arguments between processings.
   We use fork and a global variable to copy arguments between processings to avoid the expensive serialization when transferring TVM objects.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.h
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.h
+ * \brief The Ansor computational graph and related program analyses.

Review comment:
       Improve the docstring by using my comments on the python file




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.

Review comment:
       More explanation 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] MarisaKirisame commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,141 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute + ". Expect a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(_ffi_api.ComputeDAGGetInitState(self), self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply transform steps according to the history of a State.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):

Review comment:
       I will just call this codegen.

##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.

Review comment:
       this is weird. why do you return string and parse it again, instead of just returning tvm objects that represent the schedule?

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages
+        self.stage_id_map = {}    # A dict maps operation to stage id
+        self._update_stage_id_map()
+
+    @property
+    def stages(self):
+        """
+        Returns
+        -------
+        stages : List[Stage]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return self.stages_cache
+
+    @property
+    def stage_ops(self):
+        """
+        Returns
+        -------
+        ops: List[Operation]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return [stage.op for stage in self.stages_cache]
+
+    def reorder(self, stage, order):
+        """ Schedule primitive corresponds to te.reorder.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be reordered, can be a Stage order index, Stage operation or stage
+            output tensor.
+        order : List[Iterator]
+            Iterators in the expected order
+        """
+        stage_id = self._resolve_stage_id(stage)
+
+        self.state_object = _ffi_api.StateReorder(self.state_object, stage_id, order)
+        self._clear_cache()
+
+    def split(self, stage, iterator, lengths, inner_to_outer=True):
+        """ Schedule primitive corresponds to te.split.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be split, can be a Stage order index, Stage operation or stage
+            output tensor.
+        iterator : Iterator
+            The iterator to split

Review comment:
       ```suggestion
               The iterator to split upon
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """

Review comment:
       what program? in a compiler everything is a program. be more specific about what step this is.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1

Review comment:
       why int instead of bool?

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.

Review comment:
       15 seconds or miliseconds or minutes?

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.

Review comment:
       Optional[int] with None to mean read all lines.
   

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    if not GLOBAL_BUILD_ARGUMENTS:
+        raise ValueError("GLOBAL_BUILD_ARGUMENTS not found")
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=broad-except
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=broad-except
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose == 1:
+            if error_no == MeasureErrorNo.NO_ERROR:
+                print(".", end="")
+            else:
+                print(".E", end="")  # Build error
+        return filename, args, error_no, error_msg, time.time() - tic
+
+    res = call_func_with_timeout(timeout, timed_func)
+    if isinstance(res, TimeoutError):
+        if verbose == 1:
+            print(".T", end="")  # Build timeout
+        res = None, [], MeasureErrorNo.BUILD_TIMEOUT, None, timeout
+
+    return res
+
+
+@tvm._ffi.register_func("ansor.local_builder.build")
+def local_builder_build(inputs, timeout, n_parallel, build_func, verbose):
+    """ Local builder build function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    global GLOBAL_BUILD_ARGUMENTS
+    GLOBAL_BUILD_ARGUMENTS = (inputs, build_func, timeout, verbose)
+
+    pool = NoDaemonPool(n_parallel)
+    tuple_res = pool.map(local_build_worker, range(len(inputs)))
+    pool.terminate()
+    pool.join()
+    del pool
+
+    results = []
+    for res in tuple_res:
+        results.append(BuildResult(*res))
+
+    return results
+
+@tvm._ffi.register_func("ansor.local_runner.run")
+def local_run(inputs, build_results, timeout, number, repeat, min_repeat_ms, cooldown_interval,
+              verbose):
+    """ Local runner run function. """

Review comment:
       ```suggestion
       """ Execute Local runner. """
   ```

##########
File path: src/ansor/auto_schedule.cc
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.cc
+ * \brief The user interface of the Ansor auto-scheduler.
+ */
+
+#include "auto_schedule.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(TuningOptionsNode);
+
+TuningOptions::TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round,
+                             int verbose, ProgramBuilder builder, ProgramRunner runner,
+                             Array<MeasureCallback> measure_callbacks,
+                             Array<SearchCallback> pre_search_callbacks) {
+  auto node = make_object<TuningOptionsNode>();
+  node->num_measure_trials = num_measure_trials;
+  node->early_stopping = early_stopping;
+  node->num_measures_per_round = num_measures_per_round;
+  node->verbose = verbose;
+  node->builder = std::move(builder);
+  node->runner = std::move(runner);
+  node->measure_callbacks = std::move(measure_callbacks);
+  node->pre_search_callbacks = std::move(pre_search_callbacks);
+  data_ = std::move(node);
+}
+
+std::pair<te::Schedule, Array<te::Tensor> > AutoSchedule(SearchTask task,
+                                                         SearchPolicy search_policy,
+                                                         TuningOptions tuning_options) {
+  // Create a ProgramMeasurer to handle the schedule build and performance measure
+  ProgramMeasurer measurer =
+      ProgramMeasurer(tuning_options->builder, tuning_options->runner,
+                      tuning_options->measure_callbacks, tuning_options->verbose);
+  // Search for the best schedule
+  State state = search_policy->Search(
+      task, tuning_options->num_measure_trials, tuning_options->early_stopping,

Review comment:
       why dont you just pass tuning_options around?

##########
File path: src/ansor/auto_schedule.cc
##########
@@ -0,0 +1,82 @@
+/*

Review comment:
       should this file be called auto_scheduler?

##########
File path: src/ansor/serialization.cc
##########
@@ -0,0 +1,423 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/serialization.cc
+ * \brief Json serialization format for dumping and loading tuning records.
+ */
+
+#include "serialization.h"
+
+#include <dmlc/json.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "loop_state.h"
+#include "transform_step.h"
+#include "utils.h"
+
+// Json serialization handler for MeasureInput, MeasureResult
+// (and recursively for SearchTask, State, Step, ...)
+namespace dmlc {
+namespace json {
+
+inline std::vector<int>& IntArrayToVector(std::vector<int>* out,
+                                          const ::tvm::Array<::tvm::Integer>& data) {
+  out->clear();
+  for (const auto& x : data) {
+    CHECK(x.defined());
+    out->push_back(x);
+  }
+  return *out;
+}
+
+template <>
+struct Handler<::tvm::Array<::tvm::ansor::Stage>> {
+  inline static void Write(dmlc::JSONWriter* writer,
+                           const ::tvm::Array<::tvm::ansor::Stage>& data) {
+    writer->BeginArray(false);
+    writer->EndArray();
+  }
+  inline static void Read(dmlc::JSONReader* reader, ::tvm::Array<::tvm::ansor::Stage>* data) {
+    bool s;
+    reader->BeginArray();
+    s = reader->NextArrayItem();
+    CHECK(!s);
+  }
+};
+
+template <>
+struct Handler<::tvm::Array<::tvm::ansor::Step>> {
+  inline static void Write(dmlc::JSONWriter* writer, const ::tvm::Array<::tvm::ansor::Step>& data) {
+    std::vector<int> tmp;
+    writer->BeginArray(false);
+    for (size_t i = 0; i < data.size(); ++i) {
+      writer->WriteArraySeperator();
+      writer->BeginArray(false);
+      if (auto ps = data[i].as<::tvm::ansor::ReorderStepNode>()) {
+        writer->WriteArrayItem(std::string("RE"));
+        writer->WriteArrayItem(ps->stage_id);
+        writer->WriteArrayItem(IntArrayToVector(&tmp, ps->after_ids));
+      } else if (auto ps = data[i].as<::tvm::ansor::SplitStepNode>()) {
+        writer->WriteArrayItem(std::string("SP"));
+        writer->WriteArrayItem(ps->stage_id);
+        writer->WriteArrayItem(ps->iter_id);
+        writer->WriteArrayItem(ps->extent.defined() ? ::tvm::ansor::GetIntImm(ps->extent) : 0);
+        writer->WriteArrayItem(IntArrayToVector(&tmp, ps->lengths));
+        writer->WriteArrayItem(static_cast<int>(ps->inner_to_outer));
+      } else if (auto ps = data[i].as<::tvm::ansor::FuseStepNode>()) {
+        writer->WriteArrayItem(std::string("FU"));
+        writer->WriteArrayItem(ps->stage_id);
+        writer->WriteArrayItem(IntArrayToVector(&tmp, ps->fused_ids));
+      } else {
+        LOG(FATAL) << "Invalid step: " << data[i];
+      }
+      writer->EndArray();
+    }
+    writer->EndArray();
+  }
+
+  inline static void Read(dmlc::JSONReader* reader, ::tvm::Array<::tvm::ansor::Step>* data) {
+    std::vector<int> int_list;
+    bool s, inner_to_outer;
+    std::string name, scope_name, pragma_type, ti_func_name;
+    int stage_id, iter_id, extent;
+
+    reader->BeginArray();
+    data->clear();
+    while (reader->NextArrayItem()) {
+      reader->BeginArray();
+      s = reader->NextArrayItem();
+      CHECK(s);
+      reader->Read(&name);
+      if (name == "RE") {
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&stage_id);
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&int_list);
+        ::tvm::Array<::tvm::Integer> after_ids;
+        for (const auto& i : int_list) {
+          after_ids.push_back(i);
+        }
+        data->push_back(::tvm::ansor::ReorderStep(stage_id, after_ids));
+      } else if (name == "SP") {
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&stage_id);
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&iter_id);
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&extent);
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&int_list);
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&inner_to_outer);
+        ::tvm::Array<::tvm::Integer> lengths;
+        for (const auto& i : int_list) {
+          lengths.push_back(i);
+        }
+        data->push_back(::tvm::ansor::SplitStep(
+            stage_id, iter_id, extent == 0 ? ::tvm::PrimExpr() : extent, lengths, inner_to_outer));
+      } else if (name == "FU") {
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&stage_id);
+        s = reader->NextArrayItem();
+        CHECK(s);
+        reader->Read(&int_list);
+        ::tvm::Array<::tvm::Integer> fused_ids;
+        for (const auto& i : int_list) {
+          fused_ids.push_back(i);
+        }
+        data->push_back(::tvm::ansor::FuseStep(stage_id, fused_ids));
+      } else {
+        LOG(FATAL) << "Invalid step format";
+      }
+      s = reader->NextArrayItem();
+      CHECK(!s);
+    }
+  }
+};
+
+template <>
+struct Handler<::tvm::ansor::StateNode> {
+  inline static void Write(dmlc::JSONWriter* writer, const ::tvm::ansor::StateNode& data) {
+    writer->BeginArray(false);
+    writer->WriteArrayItem(data.stages);
+    writer->WriteArrayItem(data.transform_steps);
+    writer->EndArray();
+  }
+  inline static void Read(dmlc::JSONReader* reader, ::tvm::ansor::StateNode* data) {
+    reader->BeginArray();
+    bool s;
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(&data->stages);
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(&data->transform_steps);
+    s = reader->NextArrayItem();
+    CHECK(!s);
+  }
+};
+
+template <>
+struct Handler<::tvm::ansor::SearchTaskNode> {
+  inline static void Write(dmlc::JSONWriter* writer, const ::tvm::ansor::SearchTaskNode& data) {
+    writer->BeginArray(false);
+    writer->WriteArrayItem(std::string(data.workload_key));
+    writer->WriteArrayItem(data.target->str());
+    writer->EndArray();
+  }
+  inline static void Read(dmlc::JSONReader* reader, ::tvm::ansor::SearchTaskNode* data) {
+    std::string target_str;
+    bool s;
+
+    reader->BeginArray();
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(&target_str);
+    data->workload_key = std::move(target_str);
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(&target_str);
+    data->target = ::tvm::Target::Create(target_str);
+    s = reader->NextArrayItem();
+    CHECK(!s);
+  }
+};
+
+template <>
+struct Handler<::tvm::ansor::MeasureInputNode> {
+  inline static void Write(dmlc::JSONWriter* writer, const ::tvm::ansor::MeasureInputNode& data) {
+    writer->BeginArray(false);
+    writer->WriteArrayItem(*data.task.operator->());
+    writer->WriteArrayItem(*data.state.operator->());
+    writer->EndArray();
+  }
+  inline static void Read(dmlc::JSONReader* reader, ::tvm::ansor::MeasureInputNode* data) {
+    bool s;
+    auto task_node = ::tvm::make_object<::tvm::ansor::SearchTaskNode>();
+    auto state_node = ::tvm::make_object<::tvm::ansor::StateNode>();
+    state_node->complete = true;
+
+    reader->BeginArray();
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(task_node.get());
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(state_node.get());
+    s = reader->NextArrayItem();
+    CHECK(!s);
+
+    data->task = ::tvm::ansor::SearchTask(task_node);
+    data->state = ::tvm::ansor::State(state_node);
+  }
+};
+
+template <>
+struct Handler<::tvm::ansor::MeasureResultNode> {
+  inline static void Write(dmlc::JSONWriter* writer, const ::tvm::ansor::MeasureResultNode& data) {
+    writer->BeginArray(false);
+    writer->WriteArraySeperator();
+    writer->BeginArray(false);
+    for (const auto& x : data.costs) {
+      auto pf = x.as<::tvm::tir::FloatImmNode>();
+      CHECK(pf != nullptr) << "Cost can only contain float values";
+      writer->WriteArrayItem(pf->value);
+    }
+    writer->EndArray();
+    writer->WriteArrayItem(data.error_no);
+    writer->WriteArrayItem(data.all_cost);
+    writer->WriteArrayItem(static_cast<int>((data.timestamp)));
+    writer->EndArray();
+  }
+  inline static void Read(dmlc::JSONReader* reader, ::tvm::ansor::MeasureResultNode* data) {
+    bool s;
+    std::vector<double> tmp;
+
+    reader->BeginArray();
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(&tmp);
+    data->costs.clear();
+    for (const auto& i : tmp) {
+      data->costs.push_back(::tvm::FloatImm(::tvm::DataType::Float(64), i));
+    }
+    s = reader->NextArrayItem();
+    CHECK(s);
+    reader->Read(&data->error_no);
+    s = reader->NextArrayItem();

Review comment:
       refactor two call to NextArrayItem() and Read() into a single function?

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.

Review comment:
       @merrymercy had comment about this file in python. please update here as well.

##########
File path: src/ansor/utils.h
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_ANSOR_UTILS_H_
+#define TVM_ANSOR_UTILS_H_
+
+#include <dmlc/common.h>
+#include <tvm/tir/expr.h>
+
+#include <algorithm>
+#include <deque>
+#include <exception>
+#include <future>
+#include <string>
+#include <thread>
+#include <tuple>
+#include <utility>
+#include <vector>
+
+namespace std {
+
+/*! \brief Hash function for std::pair */
+template <typename T1, typename T2>
+struct hash<std::pair<T1, T2>> {
+  std::size_t operator()(const std::pair<T1, T2>& k) const {
+    return ::dmlc::HashCombine(std::hash<T1>()(k.first), std::hash<T2>()(k.second));
+  }
+};
+
+/*! \brief Hash function for std::tuple */
+template <typename T1, typename T2, typename T3>
+struct hash<std::tuple<T1, T2, T3>> {
+  std::size_t operator()(const std::tuple<T1, T2, T3>& k) const {
+    return ::dmlc::HashCombine(
+        ::dmlc::HashCombine(std::hash<T1>()(std::get<0>(k)), std::hash<T2>()(std::get<1>(k))),
+        std::hash<T3>()(std::get<2>(k)));
+  }
+};
+
+}  // namespace std
+
+namespace tvm {
+namespace ansor {
+
+/********** Utilities for Array, std::string **********/
+/*! \brief Get the first appearance index of elements in an Array */
+template <typename T>
+inline void GetIndices(const Array<T>& array, const Array<T>& to_locate, Array<Integer>* indices) {
+  for (const auto& v : to_locate) {
+    auto it = std::find(array.begin(), array.end(), v);
+    if (it != array.end()) {
+      indices->push_back(it - array.begin());
+    } else {
+      LOG(FATAL) << "Cannot find the item";
+    }
+  }
+}
+
+/*! \brief Get the first appearance index of an element in an Array */
+template <typename T>
+inline int GetIndex(const Array<T>& array, const T& to_locate) {
+  for (size_t i = 0; i < array.size(); ++i) {
+    if (array[i] == to_locate) {
+      return i;
+    }
+  }
+  LOG(FATAL) << "Cannot find the item";
+  return -1;
+}
+
+/*! \brief Replace a sub-string to another sub-string in a string */
+inline void StrReplace(std::string* base, const std::string& from, const std::string& to) {
+  auto pos = base->find(from);
+  while (pos != std::string::npos) {
+    base->replace(pos, from.size(), to);
+    pos = base->find(from, pos + to.size());
+  }
+}
+
+/********** Utilities for TVM Containers / ByteArray **********/
+/*! \brief Compute mean of a FloatImm array */
+inline double FloatArrayMean(const Array<PrimExpr>& float_array) {
+  double sum = 0;
+  if (float_array.empty()) {
+    return 0.0;
+  }
+
+  for (const auto& x : float_array) {
+    auto floatimm = x.as<tir::FloatImmNode>();
+    CHECK(floatimm != nullptr);
+    sum += floatimm->value;
+  }
+  return sum / float_array.size();
+}
+
+/********** Other Utilities **********/
+/*! \brief  Get an int value from an Expr */
+inline int64_t GetIntImm(const PrimExpr& expr) {
+  auto pint = expr.as<IntImmNode>();
+  CHECK(pint != nullptr);
+  return pint->value;
+}
+
+/*! \brief  Compute the product of the lengths of axes */
+inline int64_t AxisLengthProd(const Array<tir::IterVar>& axes) {
+  int64_t ret = 1.0;
+  for (const auto& x : axes) {
+    if (const IntImmNode* imm = x->dom->extent.as<IntImmNode>()) {
+      ret *= imm->value;
+    } else {
+      return -1.0;

Review comment:
       optional or throw error.

##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState

Review comment:
       what do you mean

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {

Review comment:
       use option

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):

Review comment:
       ```suggestion
       if not isinstance(exp, expr.IntImm):
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1

Review comment:
       also should turn this into a single config datastructure.

##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,221 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a specific schedule for its target ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a history of transformations used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to perform further operations with the current state.
+The transform history is a sequence of TransformStep which will finally be mapped to schedule
+primitives. The steps can also be used for serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures, search policy needs to get the immediate
+loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """A stage in the compute declaration. Similar to tvm.te.schedule.Stage"""
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a history of transformations used to construct it.
+
+    Each State corresponds to a specific schedule for its target ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The target StateObject, corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original target ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages
+        self.stage_id_map = {}    # A dict maps operation to stage id
+        self._update_stage_id_map()
+
+    @property
+    def stages(self):
+        """
+        Returns
+        -------
+        stages : List[Stage]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return self.stages_cache
+
+    @property
+    def stage_ops(self):
+        """
+        Returns
+        -------
+        ops: List[Operation]
+        """
+        if not self.stages_cache:
+            self.stages_cache = self.state_object.stages
+        return [stage.op for stage in self.stages_cache]
+
+    def reorder(self, stage, order):
+        """ Schedule primitive corresponds to te.reorder.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be reordered, can be a Stage order index, Stage operation or stage
+            output tensor.
+        order : List[Iterator]
+            Iterators in the expected order
+        """
+        stage_id = self._resolve_stage_id(stage)
+
+        self.state_object = _ffi_api.StateReorder(self.state_object, stage_id, order)
+        self._clear_cache()
+
+    def split(self, stage, iterator, lengths, inner_to_outer=True):
+        """ Schedule primitive corresponds to te.split.
+
+        Parameters
+        ----------
+        stage : Union[int, Operation, Tensor]
+            The target Stage to be split, can be a Stage order index, Stage operation or stage
+            output tensor.
+        iterator : Iterator
+            The iterator to split
+        lengths: List[int]
+            The split factors
+        inner_to_outer: bool = True
+            True to use `factor` to split from inner to outer,
+            False to use `nparts` to split from outer to inner

Review comment:
       ```suggestion
               Whether the factor go from inner to outer, or from outer to inner
   ```
   

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {

Review comment:
       why not just return?

##########
File path: python/tvm/ansor/serialization.py
##########
@@ -0,0 +1,156 @@
+# 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.
+
+"""Serialization and other I/O support for tuning logs (measurement records)"""
+
+import numpy as np
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import MeasureCallback, MeasureErrorNo
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.LogToFile")
+class LogToFile(MeasureCallback):
+    """
+    A measurement callback that writes measurement records into a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name for this callback to write log to.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogToFile, filename)
+
+
+@tvm._ffi.register_object("ansor.LogReader")
+class LogReader(Object):
+    """
+    Reader of the json log file.
+
+    Parameters
+    ----------
+    filename : str = "ansor_tuning.json"
+        File name for this reader to load log from.
+    """
+    def __init__(self, filename="ansor_tuning.json"):
+        self.__init_handle_by_constructor__(_ffi_api.LogReader, filename)
+
+    def read_lines(self, max_lines=-1, skip_lines=0):
+        """ Read multiple lines from the log file.
+
+        Parameters
+        ----------
+        max_lines : int = -1
+            The maximum number of lines. -1 means to read all lines.
+        skip_lines : int = 0
+            Skip the first n lines.
+
+        Returns
+        -------
+        inputs : List[MeasureInput]
+            The MeasureInputs loaded from the log file.
+        results : List[MeasureResult]
+            The MeasureResults loaded from the log file.
+        """
+        inputs, results = _ffi_api.LogReaderReadLines(self, max_lines, skip_lines)
+        return inputs, results
+
+    def __iter__(self):
+        while True:
+            ret = _ffi_api.LogReaderReadNext(self)
+            if not ret:
+                break
+            yield ret[0], ret[1]  # (input, result)
+
+
+def load_from_file(filename):
+    """
+    Load measurement records from a file.
+
+    Parameters
+    ----------
+    filename : str
+        File name to load log from.
+
+    Returns
+    -------
+    logs : List[MeasureInput, MeasureResult]
+    """
+    return zip(*LogReader(filename).read_lines())
+
+
+def append_measure_records_to_file(filename, inputs, results):
+    """
+    Aappend measure records to file.

Review comment:
       ```suggestion
       Append measure records to file.
   ```

##########
File path: src/ansor/auto_schedule.h
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.h
+ * \brief The user interface of the Ansor auto-scheduler. This is the entry structure to get
+ * schedule search requirements from upper level (Python API), and returns a high performance
+ * schedule after search process.
+ */
+
+#ifndef TVM_ANSOR_AUTO_SCHEDULE_H_
+#define TVM_ANSOR_AUTO_SCHEDULE_H_
+
+#include <utility>
+
+#include "measure.h"
+#include "search_policy/search_policy.h"
+
+namespace tvm {
+namespace ansor {
+
+/*! \brief Tuning and measurement options. */
+class TuningOptionsNode : public Object {
+ public:
+  /*! \brief Number of total measurement trials. */
+  int num_measure_trials;
+  /*! \brief Stops early the tuning if no improvement after n measurements. */
+  int early_stopping;
+  /*! \brief The number of programs to be measured at each search round. */
+  int num_measures_per_round;
+  /*! \brief Verbosity level. 0 for silent, 1 to output information during schedule searching. */
+  int verbose;
+  /*! \brief ProgramBuilder which builds the program */
+  ProgramBuilder builder;
+  /*! \brief ProgramRunner which runs the program and measure time costs */
+  ProgramRunner runner;
+  /*! \brief MeasureCallback functions to be called after each measure batch */
+  Array<MeasureCallback> measure_callbacks;
+  /*! \brief SearchCallback functions to be called before schedule search */
+  Array<SearchCallback> pre_search_callbacks;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("num_measure_trials", &num_measure_trials);
+    v->Visit("early_stopping", &early_stopping);
+    v->Visit("num_measures_per_round", &num_measures_per_round);
+    v->Visit("verbose", &verbose);
+    v->Visit("builder", &builder);
+    v->Visit("runner", &runner);
+    v->Visit("measure_callbacks", &measure_callbacks);
+    v->Visit("pre_search_callbacks", &pre_search_callbacks);
+  }
+
+  static constexpr const char* _type_key = "ansor.TuningOptions";
+  TVM_DECLARE_FINAL_OBJECT_INFO(TuningOptionsNode, Object);
+};
+
+/*!
+ * \brief Managed reference to TuningOptionsNode.
+ * \sa TuningOptionsNode
+ */
+class TuningOptions : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor
+   * \param num_measure_trials Number of total measurement trials.
+   * \param early_stopping Stops early the tuning if no improvement after n measurements.
+   * \param num_measures_per_round The number of programs to be measured at each search round.
+   * \param verbose Verbosity level. 0 for silent, 1 to output information during schedule
+   * search.
+   * \param builder ProgramBuilder which builds the program.
+   * \param runner ProgramRunner which runs the program and measure time costs.
+   * \param measure_callbacks MeasureCallback functions to be called after each measure batch.
+   * \param pre_search_callbacks SearchCallback functions to be called before schedule search.
+   */
+  TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round, int verbose,
+                ProgramBuilder builder, ProgramRunner runner,
+                Array<MeasureCallback> measure_callbacks,
+                Array<SearchCallback> pre_search_callbacks);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(TuningOptions, ObjectRef, TuningOptionsNode);
+};
+
+/*!
+ * \brief Auto schedule search for a given compute declaration, by SearchTask.
+ * \param task The target search task.
+ * \param search_policy The search policy to be used for schedule search.
+ * \param tuning_options Tuning and measurement options.
+ * \return A `te::Schedule` and the target `te::Tensor` to be used in `tvm.lower` or `tvm.build`.
+ */
+std::pair<te::Schedule, Array<te::Tensor> > AutoSchedule(SearchTask task,

Review comment:
       we are C++11 or newer. just >>.

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.
+    number : int = 3
+        Number of measure times.
+    repeat : int = 1
+        Number of repeat times in each measure.
+    min_repeat_ms : int = 0
+        The minimum duration of one repeat in milliseconds.
+    cooldown_interval : float = 0.0
+        The cool down interval between two measurements.
+    """
+
+    def __init__(self,
+                 timeout=10,
+                 number=3,
+                 repeat=1,
+                 min_repeat_ms=0,
+                 cooldown_interval=0.0):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalRunner, timeout, number, repeat, min_repeat_ms, cooldown_interval)
+
+
+class MeasureErrorNo(object):
+    """ Error type for MeasureResult. """
+    NO_ERROR = 0              # No error
+    INSTANTIATION_ERROR = 1   # Errors happen when apply transform steps from init state
+                              # Errors happen when compiling code on host (e.g. tvm.build)
+    COMPILE_HOST = 2
+    COMPILE_DEVICE = 3        # Errors happen when compiling code on device
+                              # (e.g. OpenCL JIT on the device)
+    RUNTIME_DEVICE = 4        # Errors happen when run program on device
+    WRONG_ANSWER = 5          # Answer is wrong when compared to a reference output
+    BUILD_TIMEOUT = 6         # Timeout during compilation
+    RUN_TIMEOUT = 7           # Timeout during run
+    UNKNOWN_ERROR = 8         # Unknown error
+
+
+def make_error_msg():
+    """ Get the error message from traceback. """
+    error_msg = str(traceback.format_exc())
+    if len(error_msg) > MAX_ERROR_MSG_LEN:
+        error_msg = error_msg[:MAX_ERROR_MSG_LEN//2] + \
+            "\n...\n" + error_msg[-MAX_ERROR_MSG_LEN//2:]
+    return error_msg
+
+
+def local_build_worker(index):
+    """ Local builder function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    if not GLOBAL_BUILD_ARGUMENTS:
+        raise ValueError("GLOBAL_BUILD_ARGUMENTS not found")
+    measure_inputs, build_func, timeout, verbose = GLOBAL_BUILD_ARGUMENTS
+    assert isinstance(build_func, str)
+
+    if build_func == 'default':
+        build_func = tar.tar
+    elif build_func == 'ndk':
+        build_func = ndk.create_shared
+    else:
+        raise ValueError("Invalid build_func" + build_func)
+
+    def timed_func():
+        tic = time.time()
+        inp = measure_inputs[index]
+        task = inp.task
+
+        error_no = MeasureErrorNo.NO_ERROR
+        error_msg = None
+        args = []
+
+        try:
+            sch, args = task.compute_dag.apply_steps_from_state(
+                inp.state)
+        # pylint: disable=broad-except
+        except Exception:
+            error_no = MeasureErrorNo.INSTANTIATION_ERROR
+            error_msg = make_error_msg()
+
+        if error_no == 0:
+            dirname = tempfile.mkdtemp()
+            filename = os.path.join(
+                dirname, "tmp_func." + build_func.output_format)
+
+            try:
+                with transform.PassContext():  # todo(lmzheng): port the unroll pass
+                    func = build_module.build(
+                        sch, args, target=task.target, target_host=task.target_host)
+                func.export_library(filename, build_func)
+            # pylint: disable=broad-except
+            except Exception:
+                error_no = MeasureErrorNo.COMPILE_HOST
+                error_msg = make_error_msg()
+        else:
+            filename = ""
+
+        if verbose == 1:
+            if error_no == MeasureErrorNo.NO_ERROR:
+                print(".", end="")
+            else:
+                print(".E", end="")  # Build error
+        return filename, args, error_no, error_msg, time.time() - tic
+
+    res = call_func_with_timeout(timeout, timed_func)
+    if isinstance(res, TimeoutError):
+        if verbose == 1:
+            print(".T", end="")  # Build timeout
+        res = None, [], MeasureErrorNo.BUILD_TIMEOUT, None, timeout
+
+    return res
+
+
+@tvm._ffi.register_func("ansor.local_builder.build")
+def local_builder_build(inputs, timeout, n_parallel, build_func, verbose):
+    """ Local builder build function. """
+    # We use fork to copy arguments from a global variable.
+    # This can avoid expensive serialization of TVM IR when using multiprocessing.Pool
+    global GLOBAL_BUILD_ARGUMENTS
+    GLOBAL_BUILD_ARGUMENTS = (inputs, build_func, timeout, verbose)

Review comment:
       this seems like bad program structure. can you make this local and pass it around? you still dont have to serialize stuff as everything pass is by reference.
   what do you mean by multiprocessing tool?

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr

Review comment:
       Tuple[Expr]

##########
File path: src/ansor/utils.h
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/utils.h
+ * \brief Common utilities.
+ */
+
+#ifndef TVM_ANSOR_UTILS_H_
+#define TVM_ANSOR_UTILS_H_
+
+#include <dmlc/common.h>
+#include <tvm/tir/expr.h>
+
+#include <algorithm>
+#include <deque>
+#include <exception>
+#include <future>
+#include <string>
+#include <thread>
+#include <tuple>
+#include <utility>
+#include <vector>
+
+namespace std {
+
+/*! \brief Hash function for std::pair */
+template <typename T1, typename T2>
+struct hash<std::pair<T1, T2>> {
+  std::size_t operator()(const std::pair<T1, T2>& k) const {
+    return ::dmlc::HashCombine(std::hash<T1>()(k.first), std::hash<T2>()(k.second));
+  }
+};
+
+/*! \brief Hash function for std::tuple */
+template <typename T1, typename T2, typename T3>
+struct hash<std::tuple<T1, T2, T3>> {
+  std::size_t operator()(const std::tuple<T1, T2, T3>& k) const {
+    return ::dmlc::HashCombine(
+        ::dmlc::HashCombine(std::hash<T1>()(std::get<0>(k)), std::hash<T2>()(std::get<1>(k))),
+        std::hash<T3>()(std::get<2>(k)));
+  }
+};
+
+}  // namespace std
+
+namespace tvm {
+namespace ansor {
+
+/********** Utilities for Array, std::string **********/
+/*! \brief Get the first appearance index of elements in an Array */
+template <typename T>
+inline void GetIndices(const Array<T>& array, const Array<T>& to_locate, Array<Integer>* indices) {
+  for (const auto& v : to_locate) {
+    auto it = std::find(array.begin(), array.end(), v);
+    if (it != array.end()) {
+      indices->push_back(it - array.begin());
+    } else {
+      LOG(FATAL) << "Cannot find the item";
+    }
+  }
+}
+
+/*! \brief Get the first appearance index of an element in an Array */
+template <typename T>
+inline int GetIndex(const Array<T>& array, const T& to_locate) {
+  for (size_t i = 0; i < array.size(); ++i) {
+    if (array[i] == to_locate) {
+      return i;
+    }
+  }
+  LOG(FATAL) << "Cannot find the item";
+  return -1;
+}
+
+/*! \brief Replace a sub-string to another sub-string in a string */
+inline void StrReplace(std::string* base, const std::string& from, const std::string& to) {
+  auto pos = base->find(from);
+  while (pos != std::string::npos) {
+    base->replace(pos, from.size(), to);
+    pos = base->find(from, pos + to.size());
+  }
+}
+
+/********** Utilities for TVM Containers / ByteArray **********/
+/*! \brief Compute mean of a FloatImm array */
+inline double FloatArrayMean(const Array<PrimExpr>& float_array) {
+  double sum = 0;
+  if (float_array.empty()) {
+    return 0.0;
+  }
+
+  for (const auto& x : float_array) {
+    auto floatimm = x.as<tir::FloatImmNode>();
+    CHECK(floatimm != nullptr);
+    sum += floatimm->value;
+  }
+  return sum / float_array.size();
+}
+
+/********** Other Utilities **********/
+/*! \brief  Get an int value from an Expr */
+inline int64_t GetIntImm(const PrimExpr& expr) {
+  auto pint = expr.as<IntImmNode>();
+  CHECK(pint != nullptr);
+  return pint->value;
+}
+
+/*! \brief  Compute the product of the lengths of axes */

Review comment:
       ```suggestion
   /*! \brief Compute the product of the lengths of axes */
   ```

##########
File path: python/tvm/ansor/workload_registry.py
##########
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Workload registration and serialization.
+
+We use a json string to represent a workload (a compute dag).

Review comment:
       ```suggestion
   We use a json string to represent a workload (a computation graph).
   ```

##########
File path: python/tvm/ansor/measure.py
##########
@@ -0,0 +1,386 @@
+# 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.
+
+"""Distributed measurement infrastructure to measure the runtime costs of tensor programs
+
+These functions are responsible for building the tvm module, uploading it to
+remote devices, recording the running time costs, and checking the correctness of the output.
+
+We implement these in python to utilize python's multiprocessing and error handling
+"""
+
+import os
+import time
+import shutil
+import traceback
+import tempfile
+import multiprocessing
+
+import tvm._ffi
+from tvm.runtime import Object, module, ndarray
+from tvm.driver import build_module
+from tvm.ir import transform
+from tvm.contrib import tar, ndk
+
+from . import _ffi_api
+from .utils import get_const_tuple, NoDaemonPool, call_func_with_timeout
+
+# The maximum length of error message
+MAX_ERROR_MSG_LEN = 512
+
+# Global variables used in build function
+GLOBAL_BUILD_ARGUMENTS = None
+
+@tvm._ffi.register_object("ansor.MeasureCallback")
+class MeasureCallback(Object):
+    """ The base class of measurement callback functions. """
+
+
+@tvm._ffi.register_object("ansor.MeasureInput")
+class MeasureInput(Object):
+    """ Store the input of a measurement.
+
+    Parameters
+    ----------
+    task : SearchTask
+        The target SearchTask.
+    state : State
+        The current State to be measured.
+    """
+    def __init__(self, task, state):
+        self.__init_handle_by_constructor__(_ffi_api.MeasureInput, task, state.state_object)
+
+
+@tvm._ffi.register_object("ansor.BuildResult")
+class BuildResult(Object):
+    """ Store the result of a build.
+
+    Parameters
+    ----------
+    filename : Optional[str]
+        The filename of built binary file.
+    args : List[Tensor]
+        The arguments.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    time_cost : float
+        The time cost of build.
+    """
+    def __init__(self, filename, args, error_no, error_msg, time_cost):
+        filename = filename if filename else ""
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.BuildResult, filename, args, error_no, error_msg, time_cost)
+
+
+@tvm._ffi.register_object("ansor.MeasureResult")
+class MeasureResult(Object):
+    """ Store the results of a measurement.
+
+    Parameters
+    ----------
+    costs : List[float]
+        The time costs of execution.
+    error_no : int
+        The error code.
+    error_msg : Optional[str]
+        The error message if there is any error.
+    all_cost : float
+        The time cost of build and run.
+    timestamp : float
+        The time stamps of this measurement.
+    """
+    def __init__(self, costs, error_no, error_msg, all_cost, timestamp):
+        error_msg = error_msg if error_msg else ""
+
+        self.__init_handle_by_constructor__(
+            _ffi_api.MeasureResult, costs, error_no,
+            error_msg, all_cost, timestamp)
+
+
+@tvm._ffi.register_object("ansor.ProgramBuilder")
+class ProgramBuilder(Object):
+    """ The base class of ProgramBuilders. """
+
+    def build(self, measure_inputs, verbose=1):
+        """ Build programs and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program building.
+
+        Returns
+        -------
+        res : List[BuildResult]
+        """
+        return _ffi_api.ProgramBuilderBuild(self, measure_inputs, verbose)
+
+
+@tvm._ffi.register_object("ansor.ProgramRunner")
+class ProgramRunner(Object):
+    """ The base class of ProgramRunners. """
+
+    def run(self, measure_inputs, build_results, verbose=1):
+        """ Run measurement and return results.
+
+        Parameters
+        ----------
+        measure_inputs : List[MeasureInput]
+            A List of MeasureInput.
+        build_results : List[BuildResult]
+            A List of BuildResult to be ran.
+        verbost : int = 1
+            Verbosity level. 0 for silent, 1 to output information during program running.
+
+        Returns
+        -------
+        res : List[MeasureResult]
+        """
+        return _ffi_api.ProgramRunnerRun(self, measure_inputs, build_results, verbose)
+
+
+@tvm._ffi.register_object("ansor.LocalBuilder")
+class LocalBuilder(ProgramBuilder):
+    """ LocalBuilder use local CPU cores to build programs in parallel.
+
+    Parameters
+    ----------
+    timeout : int = 15
+        The timeout limit for each build.
+    n_parallel : int = multiprocessing.cpu_count()
+        Number of threads used to build in parallel.
+    build_func : str = 'default'
+        The name of registered build function.
+    """
+
+    def __init__(self,
+                 timeout=15,
+                 n_parallel=multiprocessing.cpu_count(),
+                 build_func='default'):
+        self.__init_handle_by_constructor__(
+            _ffi_api.LocalBuilder, timeout, n_parallel, build_func)
+
+
+@tvm._ffi.register_object("ansor.LocalRunner")
+class LocalRunner(ProgramRunner):
+    """ LocalRunner that uses local CPU/GPU to measures the time cost of programs.
+
+    Parameters
+    ----------
+    timeout : int = 10
+        The timeout limit for each run.

Review comment:
       get the unit back.

##########
File path: src/ansor/auto_schedule.cc
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/auto_schedule.cc
+ * \brief The user interface of the Ansor auto-scheduler.
+ */
+
+#include "auto_schedule.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_NODE_TYPE(TuningOptionsNode);
+
+TuningOptions::TuningOptions(int num_measure_trials, int early_stopping, int num_measures_per_round,
+                             int verbose, ProgramBuilder builder, ProgramRunner runner,
+                             Array<MeasureCallback> measure_callbacks,
+                             Array<SearchCallback> pre_search_callbacks) {
+  auto node = make_object<TuningOptionsNode>();
+  node->num_measure_trials = num_measure_trials;
+  node->early_stopping = early_stopping;
+  node->num_measures_per_round = num_measures_per_round;
+  node->verbose = verbose;
+  node->builder = std::move(builder);
+  node->runner = std::move(runner);
+  node->measure_callbacks = std::move(measure_callbacks);
+  node->pre_search_callbacks = std::move(pre_search_callbacks);
+  data_ = std::move(node);
+}
+
+std::pair<te::Schedule, Array<te::Tensor> > AutoSchedule(SearchTask task,
+                                                         SearchPolicy search_policy,
+                                                         TuningOptions tuning_options) {
+  // Create a ProgramMeasurer to handle the schedule build and performance measure
+  ProgramMeasurer measurer =
+      ProgramMeasurer(tuning_options->builder, tuning_options->runner,
+                      tuning_options->measure_callbacks, tuning_options->verbose);
+  // Search for the best schedule
+  State state = search_policy->Search(
+      task, tuning_options->num_measure_trials, tuning_options->early_stopping,

Review comment:
       it will be more extensible.

##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {
+      res = Iterator(name, Range::FromMinExtent(tosplit_min, l), it->iter_type, kNone);
+      tosplit_min = 0;
+      tosplit_extent = indexdiv(tosplit_extent + l - 1, l);
+    } else {
+      res = Iterator(name, Range(), it->iter_type, kNone);
+      tosplit_min = tosplit_extent = PrimExpr();
+    }
+    outs.push_back(std::move(res));
+  }
+
+  Range range;
+  if (tosplit_min.defined() && tosplit_extent.defined()) {
+    range = Range::FromMinExtent(tosplit_min, tosplit_extent);
+  }
+  if (inner_to_outer) {
+    outs.push_back(Iterator(it->name + ".0", range, it->iter_type, kNone));
+    // Reverse the Iterator array
+    Array<Iterator> temp(outs.rbegin(), outs.rend());
+    outs = std::move(temp);
+  } else {
+    outs.push_back(
+        Iterator(it->name + "." + std::to_string(lengths.size()), range, it->iter_type, kNone));
+  }
+
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(), stage->iters.begin() + iter_id);
+  new_iters.insert(new_iters.end(), outs.begin(), outs.end());
+  new_iters.insert(new_iters.end(), stage->iters.begin() + iter_id + 1, stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return outs;
+}
+
+Array<Iterator> State::DoSplitStep(const SplitStep& step) {
+  return DoSplitStepCommon(step->stage_id, step->iter_id, step->lengths, step->inner_to_outer);
+}
+
+Iterator State::DoFuseStep(const FuseStep& step) {
+  int stage_id = step->stage_id;
+  const Stage& stage = operator->()->stages[stage_id];
+
+  String new_name;
+  PrimExpr new_extent = 1;
+  IteratorType new_iter_type = kSpecial;
+
+  for (size_t i = 0; i < step->fused_ids.size(); ++i) {
+    if (i > 0) {
+      CHECK_EQ(step->fused_ids[i]->value, step->fused_ids[i - 1]->value + 1);
+    }
+
+    const Iterator& it = stage->iters[step->fused_ids[i]];
+    new_name = new_name + it->name + "@";
+
+    if (it->range.defined() && new_extent.defined()) {
+      new_extent = new_extent * it->range->extent;
+    } else {
+      new_extent = PrimExpr();
+    }
+
+    if (i == 0) {
+      new_iter_type = it->iter_type;
+    } else {
+      if (new_iter_type != it->iter_type) {
+        new_iter_type = kMixed;
+      }
+    }
+  }
+
+  Range range;
+  if (new_extent.defined()) {
+    range = Range::FromMinExtent(0, new_extent);
+  }
+  Iterator new_it = Iterator(new_name, range, new_iter_type, kNone);
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(),
+                   stage->iters.begin() + step->fused_ids.front());
+  new_iters.push_back(new_it);
+  new_iters.insert(new_iters.end(), stage->iters.begin() + step->fused_ids.back() + 1,
+                   stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return new_it;
+}
+
+void State::DoSteps(const ComputeDAG& dag) {
+  CHECK(operator->()->stages.size()) << "Invalid State with empty operation stages.";
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  for (const auto& step : operator->()->transform_steps) {
+    if (complete_rate >= 0 && ct++ > operator->()->transform_steps.size() * complete_rate) {
+      break;
+    }
+    if (auto ps = step.as<ReorderStepNode>()) {
+      DoReorderStep(GetRef<ReorderStep>(ps));
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      DoSplitStep(GetRef<SplitStep>(ps));
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      DoFuseStep(GetRef<FuseStep>(ps));
+    } else {
+      LOG(FATAL) << "Invalid step: " << step;
+    }
+  }
+}
+
+// Print stage to ostream
+void PrintStage(std::ostream* os, int stage_id, const StateNode* state, size_t base_indent,

Review comment:
       same for const StateNode* state. why not reference?

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.
+    This allows us to start new processings inside the worker function"""
+
+    def __init__(self, *args, **kwargs):
+        kwargs['context'] = NoDaemonContext()
+        super().__init__(*args, **kwargs)
+
+    def __reduce__(self):
+        pass
+
+
+def kill_child_processes(parent_pid, sig=signal.SIGTERM):
+    """kill all child processes recursively"""
+    try:
+        parent = psutil.Process(parent_pid)
+    except psutil.NoSuchProcess:
+        return
+    children = parent.children(recursive=True)
+    for process in children:
+        try:
+            process.send_signal(sig)
+        except psutil.NoSuchProcess:
+            return
+
+
+def call_func_with_timeout(timeout, func, args=(), kwargs=None):
+    """Call a function with timeout"""
+    def func_wrapper(que):
+        if kwargs:
+            que.put(func(*args, **kwargs))
+        else:
+            que.put(func(*args))
+
+    que = multiprocessing.Queue(2)
+    process = multiprocessing.Process(target=func_wrapper, args=(que,))
+    process.start()
+    process.join(timeout)
+
+    try:
+        res = que.get(block=False)
+    except queue.Empty:
+        res = TimeoutError()
+
+    # clean queue and process
+    kill_child_processes(process.pid)
+    process.terminate()

Review comment:
       is this code exception safe? rewrite in raii style or put in finalizer if otherwise.

##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):

Review comment:
       ```suggestion
       if not isinstance(exp, expr.IntImm):
   ```

##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.

Review comment:
       ```suggestion
    * \note To add a new transform step:
   ```

##########
File path: src/ansor/measure.h
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/measure.h
+ * \brief Distributed measurement infrastructure to measure the runtime costs of tensor programs.
+ * MeasureInput -> BuildeResult -> MeasureResult
+ */
+
+#ifndef TVM_ANSOR_MEASURE_H_
+#define TVM_ANSOR_MEASURE_H_
+
+#include <unordered_map>
+#include <utility>
+
+#include "loop_state.h"
+#include "search_task.h"
+
+namespace tvm {
+namespace ansor {
+
+class SearchPolicy;
+class MeasureInput;
+class MeasureResult;
+
+/*! \brief The error code of one measurement */
+enum MeasureErrorNO {
+  /*! \brief No error. */
+  kNoError = 0,
+  /*! \brief Errors happen when apply transform steps from init state. */
+  kInstantiationError = 1,
+  /*! \brief Errors happen when compiling code on host. (when build module) */
+  kCompileHostError = 2,
+  /*! \brief Errors happen when compiling code on device. (when load module) */
+  kCompileDeviceError = 3,
+  /*! \brief Errors happen when run program on device. */
+  kRuntimeDeviceError = 4,
+  /*! \brief Answer is wrong when compared to a reference output. */
+  kWrongAnswerError = 5,
+  /*! \brief Timeout during compilation. */
+  kBuildTimeoutError = 6,
+  /*! \brief Timeout during run. */
+  kRunTimeoutError = 7,
+  /*! \brief Unknown error. */
+  kUnknonwError = 8,
+};
+
+// Inputs and results of one measurement
+
+/*! \brief Store the input of a measurement */
+class MeasureInputNode : public Object {
+ public:
+  /*! \brief The search task. */
+  SearchTask task;
+  /*! \brief The program state to be measured. */
+  State state;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("task", &task);
+    v->Visit("state", &state);
+  }
+
+  /*! \brief Do deep copy. */
+  MeasureInput copy() const;
+
+  static constexpr const char* _type_key = "ansor.MeasureInput";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MeasureInputNode, Object);
+};
+
+/*!
+ * \brief Managed reference to MeasureInputNode.
+ * \sa MeasureInputNode
+ */
+class MeasureInput : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param task The target SearchTeask.
+   * \param state The target State.
+   */
+  MeasureInput(SearchTask task, State state);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MeasureInput, ObjectRef, MeasureInputNode);
+};
+
+/*! \brief Store the result of a build. */
+class BuildResultNode : public Object {
+ public:
+  /*! \brief The filename of built binary file. */
+  String filename;
+  /*! \brief The arguments. */
+  Array<te::Tensor> args;
+  /*! \brief The error code. (0 means no error, see MeasureErrorNO) */
+  int error_no;
+  /*! \brief The error message if there is any error. */
+  String error_msg;
+  /*! \brief The time cost of build. */
+  double time_cost;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("filename", &filename);
+    v->Visit("args", &args);
+    v->Visit("error_no", &error_no);
+    v->Visit("error_msg", &error_msg);
+    v->Visit("time_cost", &time_cost);
+  }
+
+  static constexpr const char* _type_key = "ansor.BuildResult";
+  TVM_DECLARE_FINAL_OBJECT_INFO(BuildResultNode, Object);
+};
+
+/*!
+ * \brief Managed reference to BuildResultNode.
+ * \sa BuildResultNode
+ */
+class BuildResult : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param filename The filename of built binary file.
+   * \param args The arguments.
+   * \param error_no The error code.
+   * \param error_msg The error message if there is any error.
+   * \param time_cost The time cost of build.
+   */
+  BuildResult(String filename, Array<te::Tensor> args, int error_no, String error_msg,
+              double time_cost);
+  TVM_DEFINE_OBJECT_REF_METHODS(BuildResult, ObjectRef, BuildResultNode);
+};
+
+/*! \brief Store the results of a measurement. */
+class MeasureResultNode : public Object {

Review comment:
       MeasurmentNode

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }

Review comment:
       accessing stuff take some time. why 0 instead of 1?

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;

Review comment:
       what about the function call?

##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {
+      res = Iterator(name, Range::FromMinExtent(tosplit_min, l), it->iter_type, kNone);
+      tosplit_min = 0;
+      tosplit_extent = indexdiv(tosplit_extent + l - 1, l);
+    } else {
+      res = Iterator(name, Range(), it->iter_type, kNone);
+      tosplit_min = tosplit_extent = PrimExpr();
+    }
+    outs.push_back(std::move(res));
+  }
+
+  Range range;
+  if (tosplit_min.defined() && tosplit_extent.defined()) {
+    range = Range::FromMinExtent(tosplit_min, tosplit_extent);
+  }
+  if (inner_to_outer) {
+    outs.push_back(Iterator(it->name + ".0", range, it->iter_type, kNone));
+    // Reverse the Iterator array
+    Array<Iterator> temp(outs.rbegin(), outs.rend());
+    outs = std::move(temp);
+  } else {
+    outs.push_back(
+        Iterator(it->name + "." + std::to_string(lengths.size()), range, it->iter_type, kNone));
+  }
+
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(), stage->iters.begin() + iter_id);
+  new_iters.insert(new_iters.end(), outs.begin(), outs.end());
+  new_iters.insert(new_iters.end(), stage->iters.begin() + iter_id + 1, stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return outs;
+}
+
+Array<Iterator> State::DoSplitStep(const SplitStep& step) {
+  return DoSplitStepCommon(step->stage_id, step->iter_id, step->lengths, step->inner_to_outer);
+}
+
+Iterator State::DoFuseStep(const FuseStep& step) {
+  int stage_id = step->stage_id;
+  const Stage& stage = operator->()->stages[stage_id];
+
+  String new_name;
+  PrimExpr new_extent = 1;
+  IteratorType new_iter_type = kSpecial;
+
+  for (size_t i = 0; i < step->fused_ids.size(); ++i) {
+    if (i > 0) {
+      CHECK_EQ(step->fused_ids[i]->value, step->fused_ids[i - 1]->value + 1);
+    }
+
+    const Iterator& it = stage->iters[step->fused_ids[i]];
+    new_name = new_name + it->name + "@";
+
+    if (it->range.defined() && new_extent.defined()) {
+      new_extent = new_extent * it->range->extent;
+    } else {
+      new_extent = PrimExpr();
+    }
+
+    if (i == 0) {
+      new_iter_type = it->iter_type;
+    } else {
+      if (new_iter_type != it->iter_type) {
+        new_iter_type = kMixed;
+      }
+    }
+  }
+
+  Range range;
+  if (new_extent.defined()) {
+    range = Range::FromMinExtent(0, new_extent);
+  }
+  Iterator new_it = Iterator(new_name, range, new_iter_type, kNone);
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(),
+                   stage->iters.begin() + step->fused_ids.front());
+  new_iters.push_back(new_it);
+  new_iters.insert(new_iters.end(), stage->iters.begin() + step->fused_ids.back() + 1,
+                   stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return new_it;
+}
+
+void State::DoSteps(const ComputeDAG& dag) {
+  CHECK(operator->()->stages.size()) << "Invalid State with empty operation stages.";
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  for (const auto& step : operator->()->transform_steps) {
+    if (complete_rate >= 0 && ct++ > operator->()->transform_steps.size() * complete_rate) {
+      break;
+    }
+    if (auto ps = step.as<ReorderStepNode>()) {
+      DoReorderStep(GetRef<ReorderStep>(ps));
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      DoSplitStep(GetRef<SplitStep>(ps));
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      DoFuseStep(GetRef<FuseStep>(ps));
+    } else {
+      LOG(FATAL) << "Invalid step: " << step;
+    }
+  }
+}
+
+// Print stage to ostream
+void PrintStage(std::ostream* os, int stage_id, const StateNode* state, size_t base_indent,

Review comment:
       why not &? other ppl use ostream& rather then ostream*.

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*> > edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);

Review comment:
       use option




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,207 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined precise or heuristic rules to generate several
+initial schedules. Based on these initial starting points, we perform `Performance Tuning` which
+uses cost model based evolutionary search to select schedules with the best performance.
+
+Candidate schedules are measured against the specific hardware target.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware used to guide the search process of SearchPolicy.
+
+    TODO(jcf94): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The computation information and hardware parameters for a specific schedule search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for the target compute declaration.
+    workload_key : str
+        The workload key for the target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class of search policies. """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuningOptions")
+class TuningOptions(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.

Review comment:
       ```suggestion
         The number of schedules to be measured at each search round.
         The whole schedule search process will try `num_measure_trials` in several rounds.
   ```
   
   Be consistent with programs, schedules, and states.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,153 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import hashlib
+
+import tvm._ffi
+from tvm.runtime import Object
+from tvm.te import PlaceholderOp, ComputeOp
+
+from .loop_state import State, StateObject
+from .utils import get_const_tuple
+from .workload_registry import workload_key_to_tensors
+
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    The Ansor computational graph and related program analyses.
+
+    We convert a compute declaration described by `tvm.compute` (could be a single operator or a
+    subgraph) to a ComputeDAG. It keeps the input/output tensors of the target compute declaration,
+    a list of all related operations in topo order as well as a set of analyses over each operation
+    stage (e.g. the total float operation count, consumer/producer relations of each operation
+    stage, whether a operation stage should be tiled/compute inlined ...). These analyses can
+    help the search policy to do some specific decisions during schedule search process.
+
+    ComputeDAG is also responsible for the interaction between Ansor LoopState and TVM schedule
+    (e.g. applying the LoopState transform steps to TVM schedule, providing LoopState with extra
+    information get from TVM schedule ...).
+
+    Parameters
+    ----------
+    compute : Union[List[Tensor], str]
+        `Tensor`s or workload key for a compute declaration.
+    """
+    def __init__(self, compute):
+        if isinstance(compute, str):
+            compute = workload_key_to_tensors(compute)
+        elif isinstance(compute, list):
+            for item in compute:
+                if not isinstance(item, tvm.te.Tensor):
+                    raise ValueError("The input of ComputeDAG should be a list of Tensor")
+        else:
+            raise ValueError("Invalid compute: " + compute +
+                             " . `ComputeDAG` expects a string or list of Tensor")
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, compute)
+
+    def get_init_state(self):
+        """ Get the init state of this ComputeDAG.
+
+        Returns
+        -------
+        state : State
+            The initial State without any transform steps.
+        """
+        return State(self.init_state, self)
+
+    def apply_steps_from_state(self, state):
+        """
+        Apply the history transform steps of a State to TVM schedule.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+            A `te.schedule` and the target `te.Tensor`s to be used in `tvm.lower` or `tvm.build`
+        """
+        state_obj = state if isinstance(state, StateObject) else state.state_object
+        return _ffi_api.ComputeDAGApplyStepsFromState(self, state_obj)
+
+    def print_python_code_from_state(self, state):
+        """
+        Print transform steps in the history of a State as TVM's python schedule primitive.
+
+        Parameters
+        ----------
+        state : Union[State, StateObject]
+            The target state to be applied to TVM schedule.
+
+        Returns
+        -------
+        str : Str
+            The Python schedule code.

Review comment:
       This is just for debug




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/transform_step.h
##########
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/transform_step.h
+ * \brief Transformation steps. For each schedule primitive, there is a corresponding transform
+ * step. The implementation of each step consists of 2 parts:
+ * - transform_step.cc: How each step interact with TVM system
+ * - loop_state.cc:     How each step reflect on LoopState
+ *
+ * \note Adding a new transform step.
+ * Take fuse step for example:
+ * 1. Define class `FuseStepNode`, `FuseStep` in `transform_steps.h`, and implement its construction
+ *    function `FuseStep::FuseStep(...)` in `transform_steps.cc`
+ * 2. Implement `FuseStepNode::ApplyToSchedule` and `FuseStepNode::PrintAsPythonAPI`.
+ *    - In these two functions you need to lower this step with tvm's te schedule API
+ * 3. Implement `State::fuse` and `State::DoFuseStep`.
+ *    - In these two functions you need to incrementally update all data structures in State with
+ *      CopyOnWrite style
+ * 4. Add you step to `ComputeDAG::ReplaySteps` and make sure it works.
+ * 5. Add serialization support in `struct Handler<Array<::tvm::ansor::Step> >`
+ *    in `serialization.cc`.
+ * 6. Add hash support in `struct hash<::tvm::ansor::Step>`. (search for this function in this file)
+ * 7. Add its corresponding Python API to `loop_state.py` and necessary unit test.
+ */
+
+#ifndef TVM_ANSOR_TRANSFORM_STEP_H_
+#define TVM_ANSOR_TRANSFORM_STEP_H_
+
+#include <dmlc/common.h>
+#include <tvm/node/node.h>
+#include <tvm/te/schedule.h>
+
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+typedef Map<tvm::te::Stage, Array<tir::IterVar>, ObjectHash, ObjectEqual> StageToAxesMap;
+
+/*!
+ * \brief The base class for a transformation step. Each step has its corresponding tvm.te
+ * schedule primitives.
+ */
+class StepNode : public Object {
+ public:
+  /*! \brief The index of the target stage. */
+  int stage_id;
+
+  static constexpr const char* _type_key = "ansor.Step";
+  TVM_DECLARE_BASE_OBJECT_INFO(StepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StepNode.
+ * \sa StepNode
+ */
+class Step : public ObjectRef {
+ public:
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(Step, ObjectRef, StepNode);
+};
+
+/*! \brief Reorder step that corresponds to te::Stage::reorder */
+class ReorderStepNode : public StepNode {
+ public:
+  /*!
+   * \brief The iterator ids after reorder.
+   * This array should specify the order of all iterators.
+   */
+  Array<PrimExpr> after_ids;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   */
+  void ApplyToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.ReorderStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(ReorderStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to ReorderStepNode.
+ * \sa ReorderStepNode
+ */
+class ReorderStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param after_ids The index of the iterators after reorder.
+   */
+  ReorderStep(int stage_id, const Array<PrimExpr>& after_ids);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(ReorderStep, Step, ReorderStepNode);
+};
+
+/*!
+ * \brief Split step that corresponds to te::Stage::split with additional
+ *  support of multiple-level of factors
+ */
+class SplitStepNode : public StepNode {
+ public:
+  /*! \brief The id of the iter to split. */
+  int iter_id;
+  /*! \brief The extent length of the axis to split. */
+  PrimExpr extent;
+  /*! \brief The split factors. */
+  Array<PrimExpr> lengths;
+  /*!
+   * \brief If true, the `lengths` denote the lengths of iterators
+   * from inner level to outer level
+   */
+  bool inner_to_outer;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return The iterator results after split.
+   */
+  Array<tir::IterVar> ApplyToSchedule(Array<te::Stage>* stages,
+                                      StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.SplitStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(SplitStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to SplitStepNode.
+ * \sa SplitStepNode
+ */
+class SplitStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param extent The index of the target iterator.
+   * \param lengths The extent length of the axis to split.
+   * \param inner_to_outer The split direction.
+   */
+  SplitStep(int stage_id, int iter_id, PrimExpr extent, const Array<PrimExpr>& lengths,
+            bool inner_to_outer);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(SplitStep, Step, SplitStepNode);
+};
+
+/*! \brief Fuse step that corresponds to te::Stage::fuse */
+class FuseStepNode : public StepNode {
+ public:
+  /*! \brief The ids of iterators to fuse. */
+  Array<PrimExpr> fused_ids;
+
+  /*!
+   * \brief Apply the current state to tvm.schedule
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return The iterator result after fuse.
+   */
+  tir::IterVar ApplyToSchedule(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  /*!
+   * \brief Print step as equivalent python schedule API.
+   * \param stages A pointer to `te::Stage` vector.
+   * \param stage_to_axes A pointer to StageToAxesMap.
+   * \return Python schedule code.
+   */
+  String PrintAsPythonAPI(Array<te::Stage>* stages, StageToAxesMap* stage_to_axes) const;
+
+  static constexpr const char* _type_key = "ansor.FuseStep";
+  TVM_DECLARE_FINAL_OBJECT_INFO(FuseStepNode, Object);
+};
+
+/*!
+ * \brief Managed reference to FuseStepNode.
+ * \sa FuseStepNode
+ */
+class FuseStep : public Step {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param stage_id The index of the target stage.
+   * \param fused_ids The index of the target iterators to be fused.
+   */
+  FuseStep(int stage_id, const Array<PrimExpr>& fused_ids);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(FuseStep, Step, FuseStepNode);
+};
+
+}  // namespace ansor
+}  // namespace tvm
+
+// Hash and equal function for Step
+namespace std {
+
+/*! \brief The hash function of each transform step. */
+template <>
+struct hash<::tvm::ansor::Step> {

Review comment:
       Emm ... Seems that's a good idea, I think the current implementation is due to some historical issues, I'll try that way.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");

Review comment:
       Oh, this is a debug hack for some ablation study in the paper. We should delete this.

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");

Review comment:
       Oh, this is a hack for some ablation study in the paper. We should delete this.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+"""Common utilities for ansor"""
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The target function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : tvm.Expr or int
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, (expr.IntImm)):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, (expr.IntImm)):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : tuple of Expr
+        The input.
+
+    Returns
+    -------
+    out_tuple : tuple of int
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret
+
+
+class NoDaemonProcess(multiprocessing.Process):
+    @property
+    def daemon(self):
+        return False
+
+    @daemon.setter
+    def daemon(self, value):
+        pass
+
+
+class NoDaemonContext(type(multiprocessing.get_context())):
+    Process = NoDaemonProcess
+
+
+class NoDaemonPool(multiprocessing.pool.Pool):
+    """A no daemon pool version of multiprocessing.Pool.

Review comment:
       @MarisaKirisame This class inherits the builtin class in python and only sets the `daemon` property to false. Do you have a better name?
   
   @yangjunpro This is not related to synchronous or asynchronous pool




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/search_policy/search_policy.cc
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.cc
+ * \brief The base class of search policies.
+ */
+
+#include "search_policy.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(SearchCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(SearchPolicyNode);
+
+void SearchPolicyNode::RunCallbacks(const Array<SearchCallback>& callbacks) {
+  if (callbacks.defined() && callbacks.size()) {
+    for (const auto& callback : callbacks) {
+      callback->Callback(this);
+    }
+  }
+}

Review comment:
       Use `tvm::runtime::Optional` to indicate nullable value.
   
   ```suggestion
   void SearchPolicyNode::RunCallbacks(const Optional<Array<SearchCallback>>& callbacks) {
     if (callbacks.defined()) {
       for (const auto& callback : callbacks.value()) {
         callback->Callback(this);
       }
     }
   }
   ```

##########
File path: src/ansor/record.cc
##########
@@ -0,0 +1,423 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/record.cc
+ * \brief Json serialization format for dumping and loading tuning records.
+ */
+
+#include "record.h"
+
+#include <dmlc/json.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "loop_state.h"
+#include "transform_step.h"
+#include "utils.h"
+
+// Json serialization handler for MeasureInput, MeasureResult
+// (and recursively for SearchTask, State, Step, ...)
+namespace dmlc {
+namespace json {
+
+inline std::vector<int>& IntArrayToVector(std::vector<int>* out,
+                                          const ::tvm::Array<::tvm::Integer>& data) {
+  out->clear();
+  for (const auto& x : data) {
+    CHECK(x.defined());
+    out->push_back(x);
+  }
+  return *out;
+}

Review comment:
       I think it is okay to define `out` inside this function and return it, don't have to pass `out` in. Copy elision can handle it properly

##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;

Review comment:
       ```suggestion
     Optional<PrimExpr> tosplit_min, tosplit_extent;
   ```

##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {
+      res = Iterator(name, Range::FromMinExtent(tosplit_min, l), it->iter_type, kNone);
+      tosplit_min = 0;
+      tosplit_extent = indexdiv(tosplit_extent + l - 1, l);
+    } else {
+      res = Iterator(name, Range(), it->iter_type, kNone);
+      tosplit_min = tosplit_extent = PrimExpr();
+    }
+    outs.push_back(std::move(res));
+  }
+
+  Range range;
+  if (tosplit_min.defined() && tosplit_extent.defined()) {
+    range = Range::FromMinExtent(tosplit_min, tosplit_extent);
+  }
+  if (inner_to_outer) {
+    outs.push_back(Iterator(it->name + ".0", range, it->iter_type, kNone));
+    // Reverse the Iterator array
+    Array<Iterator> temp(outs.rbegin(), outs.rend());
+    outs = std::move(temp);
+  } else {
+    outs.push_back(
+        Iterator(it->name + "." + std::to_string(lengths.size()), range, it->iter_type, kNone));
+  }
+
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(), stage->iters.begin() + iter_id);
+  new_iters.insert(new_iters.end(), outs.begin(), outs.end());
+  new_iters.insert(new_iters.end(), stage->iters.begin() + iter_id + 1, stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return outs;
+}
+
+Array<Iterator> State::DoSplitStep(const SplitStep& step) {
+  return DoSplitStepCommon(step->stage_id, step->iter_id, step->lengths, step->inner_to_outer);
+}
+
+Iterator State::DoFuseStep(const FuseStep& step) {
+  int stage_id = step->stage_id;
+  const Stage& stage = operator->()->stages[stage_id];
+
+  String new_name;
+  PrimExpr new_extent = 1;
+  IteratorType new_iter_type = kSpecial;
+
+  for (size_t i = 0; i < step->fused_ids.size(); ++i) {
+    if (i > 0) {
+      CHECK_EQ(step->fused_ids[i]->value, step->fused_ids[i - 1]->value + 1);
+    }
+
+    const Iterator& it = stage->iters[step->fused_ids[i]];
+    new_name = new_name + it->name + "@";
+
+    if (it->range.defined() && new_extent.defined()) {
+      new_extent = new_extent * it->range->extent;
+    } else {
+      new_extent = PrimExpr();
+    }
+
+    if (i == 0) {
+      new_iter_type = it->iter_type;
+    } else {
+      if (new_iter_type != it->iter_type) {
+        new_iter_type = kMixed;
+      }
+    }
+  }
+
+  Range range;
+  if (new_extent.defined()) {
+    range = Range::FromMinExtent(0, new_extent);
+  }
+  Iterator new_it = Iterator(new_name, range, new_iter_type, kNone);
+  Array<Iterator> new_iters;
+  new_iters.insert(new_iters.end(), stage->iters.begin(),
+                   stage->iters.begin() + step->fused_ids.front());
+  new_iters.push_back(new_it);
+  new_iters.insert(new_iters.end(), stage->iters.begin() + step->fused_ids.back() + 1,
+                   stage->iters.end());
+
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(stage_id, Stage(stage->op, stage->op_type, std::move(new_iters),
+                                     stage->compute_at, stage->attrs));
+
+  return new_it;
+}
+
+void State::DoSteps(const ComputeDAG& dag) {

Review comment:
       Is `dag` unused?

##########
File path: src/ansor/search_policy/search_policy.cc
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/search_policy/search_policy.cc
+ * \brief The base class of search policies.
+ */
+
+#include "search_policy.h"
+
+#include <tvm/runtime/registry.h>
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(SearchCallbackNode);
+TVM_REGISTER_OBJECT_TYPE(SearchPolicyNode);

Review comment:
       Same here

##########
File path: src/ansor/loop_state.h
##########
@@ -0,0 +1,381 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.h
+ * \brief The definition of the "state" in search.
+ *
+ * Each LoopState corresponds to a schedule for its ComputeDAG.
+ * A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+ * construct the loop structure.
+ * The loop structure keeps a preview of how the schedule will finally look like after lowering the
+ * current state (e.g. number of iterators, the extent of each iterator, the compute_at locations
+ * ...).
+ * During the schedule search process, the loop structure can provide search policy with necessary
+ * information on how to manipulate the current state.
+ * The transform history is a sequence of `TransformStep` which will finally be mapped to TVM
+ * schedule primitives. The steps can also be used for the serialization of a state.
+ *
+ * The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+ * We don't use the existing TVM IR but to extend a new structure on it is because:
+ * 1. We want fast incremental change to the loop structures. The search policy needs to get the
+ * immediate loop structures update rather than after TVM lowering;
+ * 2. We want serializable transform history for replay, backtracking, and mutation;
+ * 3. We may create some macro schedule primitives that represent the combination of several
+ * TVM schedule primitives.
+ *
+ * When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+ * Since we share a lot of common objects during search, the transformation is implemented in
+ * copy on write style. All objects are immutable, which is similar to TVM IR.
+ */
+
+#ifndef TVM_ANSOR_LOOP_STATE_H_
+#define TVM_ANSOR_LOOP_STATE_H_
+
+#include <tvm/runtime/container.h>
+
+#include <functional>
+
+#include "transform_step.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+class ComputeDAG;
+
+/*! \brief The type of a stage. */
+enum StageType {
+  /*! \brief A placeholder stage. */
+  kPlaceholder = 0,
+  /*! \brief A compute stage. */
+  kCompute = 1
+};
+
+/*! \brief The type of compute location. */
+enum ComputeAtType {
+  /*! \brief Compute at root. */
+  kRoot = 0,
+  /*! \brief Compute inlined. */
+  kInlined = 1,
+  /*! \brief Compute at some iterator. */
+  kIter = 2,
+};
+
+/*! \brief The type of an iterator. */
+enum IteratorType {
+  /*! \brief Spatial iterator. */
+  kSpace = 0,
+  /*! \brief Reduction iterator. */
+  kReduce = 1,
+  /*! \brief Fused spatial and reduction iterator. */
+  kMixed = 2,
+  /*! \brief Special iterator. (e.g. virtual root iterator) */
+  kSpecial = 3
+};
+
+/*! \brief The type of an iterator's annotation. */
+enum IteratorAnnotation {
+  /*! \brief This iterator has no annotation. */
+  kNone = 0,
+  /*! \brief This iterator has been unrolled. */
+  kUnroll = 1,
+  /*! \brief This iterator has been vectorized. */
+  kVectorize = 2,
+  /*! \brief This iterator has been paralleld. */
+  kParallel = 3,
+  /*! \brief This iterator has been bind to vthread. */
+  kVThread = 4,
+  /*! \brief This iterator has been bind to blockIdx.x. */
+  kBlockX = 5,
+  /*! \brief This iterator has been bind to threadIdx.x. */
+  kThreadX = 6,
+  /*! \brief This iterator has been bind to blockIdx.y. */
+  kBlockY = 7,
+  /*! \brief This iterator has been bind to threadIdx.y. */
+  kThreadY = 8,
+  /*! \brief This iterator has been mapped with a tensorize intrinsic. */
+  kTensorized = 9
+};
+
+/*!
+ * \brief A for loop iterator
+ * Similar to tvm::IterVar in `include/tvm/tir/expr.h`
+ */
+class IteratorNode : public Object {
+ public:
+  /*! \brief The name of this iterator. */
+  String name;
+  /*! \brief The range of this iterator. */
+  Range range;
+  /*! \brief The iterator type of this iterator. */
+  IteratorType iter_type;
+  /*! \brief The annotation type of this iterator. */
+  IteratorAnnotation annotation;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("range", &range);
+  }
+
+  static constexpr const char* _type_key = "ansor.Iterator";
+  TVM_DECLARE_FINAL_OBJECT_INFO(IteratorNode, Object);
+};
+
+/*!
+ * \brief Managed reference to IteratorNode.
+ * \sa IteratorNode
+ */
+class Iterator : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param name The name of this iterator.
+   * \param range The range of this iterator.
+   * \param iter_type The iterator type of this iterator.
+   * \param annotation The annotation type of this iterator.
+   */
+  Iterator(String name, Range range, IteratorType iter_type, IteratorAnnotation annotation);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(Iterator, ObjectRef, IteratorNode);
+};
+
+/*! \brief Stage-level attributes. */
+struct StageAttributes {
+  /*! \brief The maximum steps for the pragma `auto_unroll_max_step`. */
+  int auto_unroll_max_step;
+  /*! \brief The storage offset for the schedule primitive `storage_align`. */
+  int storage_offset;
+};
+
+/*!
+ * \brief A op stage in the compute declaration.
+ * Similar to te::Stage in `include/schedule.h`.
+ */
+class StageNode : public Object {
+ public:
+  /*! \brief The operator of this stage */
+  te::Operation op;
+  /*! \brief The type of this stage. */
+  StageType op_type;
+  /*! \brief The iterators in this stage. */
+  Array<Iterator> iters;
+  /*! \brief The compute location of this stage. */
+  ComputeAtType compute_at;
+  /*! \brief Other stage-level attributes. */
+  StageAttributes attrs;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("op", &op);
+    v->Visit("iters", &iters);
+  }
+
+  static constexpr const char* _type_key = "ansor.Stage";
+  TVM_DECLARE_FINAL_OBJECT_INFO(StageNode, Object);
+};
+
+/*!
+ * \brief Managed reference to StageNode.
+ * \sa StageNode
+ */
+class Stage : public ObjectRef {
+ public:
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   */
+  explicit Stage(te::Operation op);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (copy)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters, ComputeAtType compute_at,
+        StageAttributes attrs);
+  /*!
+   * \brief The constructor.
+   * \param op A `te::Operation`.
+   * \param op_type The stage type of this op.
+   * \param iters The iterators of this op. (move)
+   * \param compute_at The compute at type of this op.
+   * \param attrs Other stage-level attributes.
+   */
+  Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,

Review comment:
       Moving semantics are not really useful in this case: `Array<>` is a simple ref counted pointer.

##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();

Review comment:
       ```suggestion
       tosplit_min = NullOpt;
       tosplit_extent = NullOpt;
   ```

##########
File path: src/ansor/loop_state.cc
##########
@@ -0,0 +1,447 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/loop_state.cc
+ * \brief An lightweight IR (intermediate representation) for loop structures.
+ * see ansor/loop_state.h for more explanation.
+ */
+
+#include "loop_state.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+
+#include <utility>
+
+#include "transform_step.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+TVM_REGISTER_OBJECT_TYPE(StepNode);
+TVM_REGISTER_NODE_TYPE(StageNode);
+TVM_REGISTER_NODE_TYPE(StateNode);
+TVM_REGISTER_NODE_TYPE(IteratorNode);
+
+/********** Iterator **********/
+Iterator::Iterator(String name, Range range, IteratorType iter_type,
+                   IteratorAnnotation annotation) {
+  auto node = make_object<IteratorNode>();
+  node->name = std::move(name);
+  node->range = std::move(range);
+  node->iter_type = iter_type;
+  node->annotation = annotation;
+  data_ = std::move(node);
+}
+
+/********** Stage **********/
+Stage::Stage(te::Operation op) {
+  auto node = make_object<StageNode>();
+  if (op->IsInstance<te::ComputeOpNode>()) {
+    node->op_type = kCompute;
+    auto* pop = op.as<te::ComputeOpNode>();
+    for (const auto& axis : pop->axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kSpace, kNone));
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      node->iters.push_back(Iterator(CleanName(axis->var->name_hint), axis->dom, kReduce, kNone));
+    }
+  } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+    node->op_type = kPlaceholder;
+  } else {
+    LOG(FATAL) << "Unsupported operator type" << op->_type_key;
+  }
+
+  node->compute_at = kRoot;
+  node->op = std::move(op);
+  node->attrs.auto_unroll_max_step = 0;
+  node->attrs.storage_offset = 0;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, const Array<Iterator>& iters,
+             ComputeAtType compute_at, StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = iters;
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+Stage::Stage(te::Operation op, StageType op_type, Array<Iterator>&& iters, ComputeAtType compute_at,
+             StageAttributes attrs) {
+  auto node = make_object<StageNode>();
+  node->op = std::move(op);
+  node->op_type = op_type;
+  node->iters = std::move(iters);
+  node->compute_at = compute_at;
+  node->attrs = attrs;
+  data_ = std::move(node);
+}
+
+/********** State **********/
+State::State(const Array<te::Operation>& ops) {
+  auto node = make_object<StateNode>();
+  for (const auto& op : ops) {
+    node->stages.push_back(Stage(op));
+  }
+  node->complete = true;
+  data_ = std::move(node);
+}
+
+/********** Schedule primitives apis for state **********/
+void State::reorder(int stage_id, const Array<Iterator>& order) {
+  const Stage& stage = operator->()->stages[stage_id];
+  CHECK_EQ(order.size(), stage->iters.size()) << "The order of all iterators "
+                                              << "should be specified";
+  Array<Integer> after_ids;
+  GetIndices(stage->iters, order, &after_ids);
+  ReorderStep step = ReorderStep(stage_id, after_ids);
+  CopyOnWrite()->transform_steps.push_back(step);
+  DoReorderStep(step);
+}
+
+Array<Iterator> State::split(int stage_id, const Iterator& it, const Array<Integer>& lengths,
+                             bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  SplitStep step =
+      SplitStep(stage_id, GetIndex(stage->iters, it),
+                it->range.defined() ? it->range->extent : PrimExpr(), lengths, inner_to_outer);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoSplitStep(step);
+}
+
+Iterator State::fuse(int stage_id, const Array<Iterator>& iters) {
+  const Stage& stage = operator->()->stages[stage_id];
+  Array<Integer> indices;
+  GetIndices(stage->iters, iters, &indices);
+  FuseStep step = FuseStep(stage_id, indices);
+  CopyOnWrite()->transform_steps.push_back(step);
+  return DoFuseStep(step);
+}
+
+/********** Step implementations for state **********/
+void State::DoReorderStep(const ReorderStep& step) {
+  const Stage& stage = operator->()->stages[step->stage_id];
+  Array<Iterator> iters;
+  for (auto x : step->after_ids) {
+    iters.push_back(stage->iters[x]);
+  }
+  StateNode* pstate = CopyOnWrite();
+  pstate->stages.Set(step->stage_id, Stage(stage->op, stage->op_type, std::move(iters),
+                                           stage->compute_at, stage->attrs));
+}
+
+// common part for DoSplitStep, DoFollowSplitStep, and DoFollowFusedSplitStep
+Array<Iterator> State::DoSplitStepCommon(int stage_id, int iter_id, const Array<Integer>& lengths,
+                                         bool inner_to_outer) {
+  const Stage& stage = operator->()->stages[stage_id];
+  const Iterator& it = stage->iters[iter_id];
+
+  PrimExpr tosplit_min, tosplit_extent;
+  if (it->range.defined()) {
+    tosplit_min = it->range->min;
+    tosplit_extent = it->range->extent;
+  } else {
+    tosplit_min = tosplit_extent = PrimExpr();
+  }
+
+  Array<Iterator> outs;
+  for (size_t i = 0; i < lengths.size(); ++i) {
+    PrimExpr l;
+    String name;
+    if (inner_to_outer) {
+      l = lengths[lengths.size() - i - 1];
+      name = it->name + "." + std::to_string(lengths.size() - i);
+    } else {
+      l = lengths[i];
+      name = it->name + "." + std::to_string(i);
+    }
+    Iterator res;
+    if (l.defined() && tosplit_min.defined() && tosplit_extent.defined()) {

Review comment:
       Does it mean some integers in `lengths` are None?




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/loop_state.py
##########
@@ -0,0 +1,223 @@
+# 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.
+# pylint: disable=unused-import
+
+"""
+The definition of the "state" in search.
+
+Each LoopState corresponds to a schedule for its ComputeDAG.
+A LoopState consists of: 1. a current loop structure; 2. a list of transformation steps used to
+construct the loop structure.
+The loop structure keeps a preview of how the schedule will finally look like after lowering the
+current state (e.g. number of iterators, the extent of each iterator, the compute_at locations ...).
+During the schedule search process, the loop structure can provide search policy with necessary
+information on how to manipulate the current state.
+The transform history is a sequence of `TransformStep` which will finally be mapped to TVM schedule
+primitives. The steps can also be used for the serialization of a state.
+
+The LoopState can be seen as a lightweight loop structure IR specifically for schedule search.
+We don't use the existing TVM IR but to extend a new structure on it is because:
+1. We want fast incremental change to the loop structures. The search policy needs to get the
+immediate loop structures update rather than after TVM lowering;
+2. We want serializable transform history for replay, backtracking, and mutation;
+3. We may create some macro schedule primitives that represent the combination of several
+TVM schedule primitives.
+
+When the search is complete, we will lower the state to TVM IR with TVM's schedule primitives.
+Since we share a lot of common objects during search, the transformation is implemented in
+copy on write style. All objects are immutable, which is similar to TVM IR.
+"""
+
+import tvm._ffi
+from tvm.te.tensor import Operation, Tensor
+from tvm.runtime import Object
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.Iterator")
+class Iterator(Object):
+    """ A loop iterator structure. """
+
+
+@tvm._ffi.register_object("ansor.Stage")
+class Stage(Object):
+    """ A stage in the compute declaration. Similar to tvm.te.schedule.Stage. """
+
+
+@tvm._ffi.register_object("ansor.State")
+class StateObject(Object):
+    """ The internal State object """
+    def __eq__(self, other):
+        return _ffi_api.StateEqual(self, other)
+
+
+class State:
+    """
+    A state in the search process. It consists of the current loop structure
+    and a list of transformation steps used to construct it.
+
+    Each State corresponds to a specific schedule for its ComputeDAG.
+
+    Parameters
+    ----------
+    state_object : StateObject
+        The StateObject corresponding to C++ internal State object.
+    dag : ComputeDAG
+        The original ComputeDAG of this State.
+
+    Notes
+    -----
+    This is a wrapper class of StateObject to deal with copy-on-write property
+    """
+    def __init__(self, state_object, dag):
+        self.state_object = state_object
+        self.compute_dag = dag
+
+        self.stages_cache = None  # A list to cache all stages

Review comment:
       Thanks, use stage cache here is becasue we need to do extra convert from C++ to python befor.
   Since the stage member of state has already been updated to Array<Stage>, this cache is no longer needed. I will remove 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] yangjunpro commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: tests/python/unittest/test_ansor_search_policy.py
##########
@@ -0,0 +1,86 @@
+# 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.
+
+"""Test search policy"""
+
+import random
+import numpy as np
+import tempfile
+
+import tvm
+from tvm import ansor
+
+from test_ansor_common import matmul_ansor_test, PropagatingThread
+
+def search_common(target="llvm", seed=random.randint(1, 1 << 30), runner='local',
+                  cost_model=None, num_measure_trials=2, params=None,
+                  pre_search_callbacks=None):
+    print("Test %s schedule search with the default search policy" % (target))
+
+    random.seed(seed)
+    N = 128
+    workload_key = ansor.make_workload_key(matmul_ansor_test, (N, N, N))
+    dag = ansor.ComputeDAG(workload_key)
+    target = tvm.target.create(target)
+    task = ansor.SearchTask(dag, workload_key, target)
+
+    with tempfile.NamedTemporaryFile() as fp:
+        log_file = fp.name
+
+        search_policy = ansor.EmptyPolicy()
+        # search_policy = ansor.SketchSearchPolicy(cost_model, params=params, seed=seed)
+        tuning_options = ansor.TuningOptions(num_measure_trials=num_measure_trials, runner=runner,
+                                             verbose=0,
+                                             measure_callbacks=[ansor.LogToFile(log_file)],
+                                             pre_search_callbacks=pre_search_callbacks)
+        sch, args = ansor.auto_schedule(task, target, search_policy=search_policy,
+                                        tuning_options=tuning_options)
+        inp, res = ansor.best_measure_pair_in_file(log_file, workload_key, target)
+
+        print("==== Python Code ====")
+        print(dag.print_python_code_from_state(inp.state))
+
+        try:
+            print("==== Lowered Stmt ====")
+            print(tvm.lower(sch, args, simple_mode=True))
+            mod = tvm.build(sch, args, target)
+
+            ctx = tvm.context(str(target), 0)
+            dtype = dag.tensors[0].dtype
+            a = tvm.nd.array(np.random.uniform(size=(N, N)).astype(dtype), ctx)
+            b = tvm.nd.array(np.random.uniform(size=(N, N)).astype(dtype), ctx)
+            c = tvm.nd.array(np.zeros((N, N), dtype=dtype), ctx)
+            mod(a, b, c)
+            tvm.testing.assert_allclose(c.asnumpy(), np.dot(
+                a.asnumpy(), b.asnumpy()), rtol=1e-5)
+            print("==== Verification passed ====")
+        except Exception:
+            raise Exception("Error encountered with seed: %d" % (seed))
+    print()
+
+
+def test_search_basic():
+    if not tvm.runtime.enabled("llvm"):
+        return

Review comment:
       why here we directly return if llvm is not enabled? I think there are still alternative backends other than LLVM? 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is design to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int
+        The max split factor for the innermost tile.
+    """
+    def __init__(self, num_cores, vector_unit_bytes, cache_line_bytes,
+                 max_unroll_vec, max_innermost_split_factor):
+        self.__init_handle_by_constructor__(_ffi_api.HardwareParams, num_cores,
+                                            vector_unit_bytes, cache_line_bytes,
+                                            max_unroll_vec, max_innermost_split_factor)
+
+
+@tvm._ffi.register_object("ansor.SearchTask")
+class SearchTask(Object):
+    """ The meta-information of a search task.
+
+    Parameters
+    ----------
+    dag : ComputeDAG
+        The ComputeDAG for target compute declaration.
+    workload_key : str
+        The workload key for target compute declaration.
+    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.
+    """
+    def __init__(self, dag, workload_key, target, target_host=None,
+                 hardware_params=None):
+        self.__init_handle_by_constructor__(_ffi_api.SearchTask, dag,
+                                            workload_key, target, target_host,
+                                            hardware_params)
+
+
+@tvm._ffi.register_object("ansor.SearchPolicy")
+class SearchPolicy(Object):
+    """ The base class for search policy  """
+
+
+@tvm._ffi.register_object("ansor.EmptyPolicy")
+class EmptyPolicy(SearchPolicy):
+    """ This is an example empty search policy which will always generate
+    the init state of target ComputeDAG.
+    """
+    def __init__(self):
+        self.__init_handle_by_constructor__(_ffi_api.EmptyPolicy)
+
+
+@tvm._ffi.register_object("ansor.TuneOption")
+class TuneOption(Object):
+    """ This controls the options of performance tuning.
+
+    Parameters
+    ----------
+    num_measure_trials: int = 0
+      The number of total schedule measure trials.
+      Ansor takes `num_measure_trials` state for measuring in total, and finally gets the best
+      schedule among them.
+      With `num_measure_trials` == 0, Ansor will do the schedule search but don't involve
+      measurement, this can be used if we want to quickly get a runnable schedule without
+      performance tuning.
+    early_stopping: int = -1
+      Stops early the tuning if no improvement get after n measurements.
+    num_measures_per_round: int = 64
+      The number of programs to be measured at each search round.
+      The whole schedule search process is designed to have several rounds to try a total
+      `num_measure_trials` schedules.
+      We have: `num_search_rounds` = `num_measure_trials` // `num_measures_per_round`
+    verbose: int = 1
+      Verbosity level. 0 for silent, 1 to output information during schedule search.
+    builder: Union[Builder, str] = 'local'
+      Builder which builds the program.
+    runner: Union[Runner, str] = 'local'
+      Runner which runs the program and measures time costs.
+    measure_callbacks: Optional[List[MeasureCallback]]
+      Callback functions called after each measure.
+      Candidates:
+        - ansor.LogToFile
+    pre_search_callbacks: Optional[List[SearchCallback]]
+      Callback functions called before the search process.
+      Candidates:
+        - ansor.PreloadMeasuredStates
+        - ansor.PreloadCustomSketchRule
+        TODO(jcf94): Add these implementation in later PRs.
+    """
+    def __init__(self, num_measure_trials=0, early_stopping=-1, num_measures_per_round=64,

Review comment:
       In my opinions, TuningOptions is already a class holding configurations related to schedule tuning stuffs, I think it might be a little bit overkill to introduce another config dict? 

##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,206 @@
+# 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.
+
+"""
+User interface for Ansor auto-scheduler.
+
+The basic schedule search process for Ansor is designed to be:
+`Program sampling` -> `Performance Tuning`.
+
+In `Program sampling`, we use some predefined or heuristic rules to generate several initial
+schedules. Based on these initial start points, we have `Performance Tuning` to apply cost model
+and evolutionary search to seek for schedules with the best performance. Candidate schedules will
+be measured in the target hardware.
+"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .compute_dag import ComputeDAG
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    TODO(...): This is considering to merge with the new Target:
+    https://discuss.tvm.ai/t/rfc-tvm-target-specification/6844
+
+    Parameters

Review comment:
       Yes, sometimes we may choose not to use all the cores available in the host system since some of the cores need to be kept for other purpose, such as data pre-processing, etc. So I think it's better to leave _num_cores_  a user-specified options with default value. 
   Regarding to _vector_unit_bytes_/_cache_line_bytes_, in my understanding they are  primitive hardware configurations, maybe we can consider to auto-detect it? 
   The principle is that we should do auto-detect as much as we can, without too much overkill. 




----------------------------------------------------------------
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] junrushao1994 commented on a change in pull request #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages.push_back(stage);
+    UpdateStageAxis(stage, &stage_to_axes);
+  }
+
+  std::stringstream ss;
+  for (const auto& stage : stages) {
+    if (stage->op->IsInstance<te::ComputeOpNode>()) {
+      for (size_t i = 0; i < stage->leaf_iter_vars.size(); ++i) {
+        ss << stage->leaf_iter_vars[i]->var->name_hint;
+        if (i != stage->leaf_iter_vars.size() - 1) {
+          ss << ", ";
+        }
+      }
+      ss << " = "
+         << "tuple(" << stage->op->name << ".op.axis)"
+         << " + "
+         << "tuple(" << stage->op->name << ".op.reduce_axis)\n";
+    }
+  }
+  // Call each step's PrintAsPythonAPI method
+  for (const auto& step : transform_steps) {
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return ss.str();
+}
+
+State ComputeDAG::InferBound(const State& state) const {
+  State ret_state;
+  StateNode* pstate;
+
+  if (state->stages.size()) {
+    ret_state = state;
+    pstate = ret_state.CopyOnWrite();
+  } else {
+    // If the input state is incomplete with empty operation stage
+    // create a new state from init_state and update it first
+    ret_state = operator->()->init_state;
+    pstate = ret_state.CopyOnWrite();
+    pstate->transform_steps = state->transform_steps;
+    ret_state.DoSteps((*this));
+  }
+
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  te::Schedule sch;
+  Array<te::Tensor> tensors;
+  // Replay steps to tvm::Schedule
+  std::tie(sch, tensors) = ApplySteps(pstate->transform_steps, &stages, &stage_to_axes);
+  sch = sch.normalize();
+  // Get bound information from TVM schedule
+  Map<IterVar, Range> bounds = te::InferBound(sch);
+
+  // Update the state bound information
+  for (size_t i = 0; i < pstate->stages.size(); ++i) {
+    const Stage& stage = pstate->stages[i];
+
+    if (stage->compute_at == kInlined) {
+      continue;
+    }
+
+    Array<Iterator> new_iters;
+    new_iters.reserve(stage->iters.size());
+    // Get bound information from schedule
+    // the StageToAxesMap is used to find the corresponding IterVar in TVM schedule result
+    for (size_t j = 0; j < stage->iters.size(); ++j) {
+      const Iterator& iter = stage->iters[j];
+      const IterVar& axis = stage_to_axes.at(stages[i])[j];
+
+      auto find_res = bounds.find(axis);
+      if (find_res != bounds.end()) {
+        new_iters.push_back(
+            Iterator(iter->name, (*find_res).second, iter->iter_type, iter->annotation));
+      } else {
+        LOG(FATAL) << "Infer bound fails";
+      }
+    }
+
+    pstate->stages.Set(
+        i, Stage(stage->op, stage->op_type, std::move(new_iters), stage->compute_at, stage->attrs));
+  }
+
+  return ret_state;
+}
+
+void ComputeDAG::InferBound(Array<State>* states) const {

Review comment:
       It's fine to just return the new states:
   
   ```suggestion
   Array<State> ComputeDAG::InferBound(const Array<State> &states) const {
   ```

##########
File path: src/ansor/compute_dag.cc
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file ansor/compute_dag.cc
+ * \brief Compute declaration graph and its related analysis tools.
+ */
+
+#include "compute_dag.h"
+
+#include <tvm/runtime/registry.h>
+#include <tvm/te/operation.h>
+#include <tvm/te/schedule.h>
+#include <tvm/te/schedule_pass.h>
+#include <tvm/tir/stmt_functor.h>
+
+#include <algorithm>
+#include <queue>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "loop_state.h"
+#include "utils.h"
+
+namespace tvm {
+namespace ansor {
+
+using namespace tvm::tir;
+
+TVM_REGISTER_NODE_TYPE(ComputeDAGNode);
+
+// Topo-sort ops from tensors according to their read-write relations.
+// Results are stored in ops
+void TopoSortOps(const Array<te::Tensor>& tensors, Array<te::Operation>* ops) {
+  std::unordered_map<const te::OperationNode*, int> degree;
+  std::unordered_map<const te::OperationNode*, std::vector<const te::OperationNode*>> edge_set;
+  std::unordered_map<const te::OperationNode*, int> priority;
+  std::unordered_set<const te::OperationNode*> visited;
+
+  // traverse to build edge_set and count degree
+  std::vector<const te::OperationNode*> stack;
+  stack.reserve(tensors.size());
+  for (const auto& x : tensors) {
+    stack.push_back(x->op.operator->());
+  }
+
+  int ct = 0;
+  while (!stack.empty()) {
+    const te::OperationNode* op = stack.back();
+    stack.pop_back();
+    if (visited.count(op)) {
+      continue;
+    }
+
+    priority[op] = ct;
+    ct++;
+    visited.insert(op);
+
+    if (op->IsInstance<te::PlaceholderOpNode>()) {
+      degree[op] = 0;
+    } else if (auto cop = GetRef<te::Operation>(op).as<te::ComputeOpNode>()) {
+      const Array<te::Tensor>& input_tensors = cop->InputTensors();
+      degree[op] = input_tensors.size();
+      for (const auto& ten : input_tensors) {
+        edge_set[ten->op.operator->()].push_back(op);
+        stack.push_back(ten->op.operator->());
+      }
+    } else {
+      LOG(FATAL) << "Unsupported op " << GetRef<te::Operation>(op);
+    }
+  }
+
+  // topo sort
+  ops->clear();
+
+  using Item = std::pair<const te::OperationNode*, int>;
+  auto cmp = [](const Item& left, const Item& right) { return left.second < right.second; };
+  std::priority_queue<Item, std::vector<Item>, decltype(cmp)> queue(cmp);
+  for (const auto& iter : degree) {
+    if (iter.second == 0) {
+      queue.push(Item(iter.first, priority[iter.first]));
+    }
+  }
+
+  ops->reserve(degree.size());
+  while (!queue.empty()) {
+    Item item = queue.top();
+    queue.pop();
+    ops->push_back(GetRef<te::Operation>(item.first));
+    for (const auto& dst : edge_set[item.first]) {
+      degree[dst] -= 1;
+      if (degree[dst] == 0) {
+        queue.push(Item(dst, priority[dst]));
+      }
+    }
+  }
+}
+
+// Estimate number of float operations in an expression
+class FlopEstimator : public ExprFunctor<double(const PrimExpr& n)> {
+ public:
+  double EstimateFlop(const Array<te::Operation>& ops) {
+    double ret = 0;
+    for (const auto& op : ops) {
+      if (auto pop = op.as<te::ComputeOpNode>()) {
+        double num_element = AxisLengthProd(pop->axis);
+        if (num_element == -1) {
+          fail = true;
+          break;
+        }
+        double op_per_element = 0;
+        for (const auto& x : pop->body) {
+          op_per_element += VisitExpr(x);
+        }
+        ret += num_element * op_per_element;
+      } else if (op->IsInstance<te::PlaceholderOpNode>()) {
+        {}  // do nothing
+      } else {
+        LOG(FATAL) << "Invalid op type " << op;
+      }
+    }
+
+    return fail ? -1 : ret;
+  }
+
+  double VisitExpr_(const ReduceNode* op) final {
+    uint64_t num_iter = 1;
+    for (const auto& x : op->axis) {
+      if (auto imm = x->dom->extent.as<IntImmNode>()) {
+        num_iter *= imm->value;
+      } else {
+        fail = true;
+        num_iter = -1;
+      }
+    }
+    double body_flop = 0;
+    for (size_t i = 0; i < op->combiner->result.size(); ++i) {
+      body_flop += VisitExpr(op->combiner->result[i]);
+      body_flop += VisitExpr(op->source[i]);
+    }
+    return num_iter * body_flop;
+  }
+
+  double VisitExpr_(const FloatImmNode* op) final { return 0.0; }
+  double VisitExpr_(const IntImmNode* op) final { return 0.0; }
+  double VisitExpr_(const ProducerLoadNode* op) final { return 0.0; }
+
+  double VisitExpr_(const CastNode* op) final { return VisitExpr(op->value); }
+  double VisitExpr_(const VarNode* op) final { return 0.0; }
+
+  double VisitExpr_(const SelectNode* op) final {
+    return VisitExpr(op->condition) +
+           std::max(VisitExpr(op->true_value), VisitExpr(op->false_value));
+  }
+
+#define VisitBinary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a) + VisitExpr(op->b); }
+#define VisitUnary(Node) \
+  double VisitExpr_(const Node* op) final { return 1.0 + VisitExpr(op->a); }
+
+  VisitBinary(AddNode);
+  VisitBinary(SubNode);
+  VisitBinary(MulNode);
+  VisitBinary(DivNode);
+  VisitBinary(ModNode);
+  VisitBinary(FloorDivNode);
+  VisitBinary(FloorModNode);
+  VisitBinary(MaxNode);
+  VisitBinary(MinNode);
+  VisitBinary(EQNode);
+  VisitBinary(NENode);
+  VisitBinary(LTNode);
+  VisitBinary(LENode);
+  VisitBinary(GTNode);
+  VisitBinary(GENode);
+  VisitBinary(AndNode);
+  VisitBinary(OrNode);
+  VisitUnary(NotNode);
+
+  double VisitExpr_(const CallNode* op) final {
+    double ret = 0.0;
+    for (const auto& x : op->args) {
+      ret += VisitExpr(x);
+    }
+    return ret;
+  }
+
+  double VisitExprDefault_(const Object* op) final {
+    fail = true;
+    return -1.0;
+  }
+
+  bool fail{false};
+};
+
+ComputeDAG::ComputeDAG(Array<te::Tensor> tensors) {
+  auto node = make_object<ComputeDAGNode>();
+  FlopEstimator estimator;
+  Array<te::Operation> ops;
+  node->tensors = std::move(tensors);
+  TopoSortOps(node->tensors, &ops);
+  node->ops = std::move(ops);
+  node->flop_ct = estimator.EstimateFlop(node->ops);
+  node->init_state = State(node->ops);
+  data_ = std::move(node);
+}
+
+// Update the te::stage to tir::IterVar axis mapping
+void UpdateStageAxis(const te::Stage& stage, StageToAxesMap* stage_to_axes) {
+  if (auto pop = stage->op.as<te::ComputeOpNode>()) {
+    Array<IterVar> axes;
+    for (const auto& axis : pop->axis) {
+      axes.push_back(axis);
+    }
+    for (const auto& axis : pop->reduce_axis) {
+      axes.push_back(axis);
+    }
+    stage_to_axes->Set(stage, std::move(axes));
+  } else if (stage->op->IsInstance<te::PlaceholderOpNode>()) {
+    {}  // do nothing on Placeholder
+  } else {
+    LOG(FATAL) << "Invalid op " << stage->op;
+  }
+}
+
+std::pair<te::Schedule, Array<te::Tensor>> ComputeDAG::ApplySteps(
+    const Array<Step>& transform_steps, Array<te::Stage>* stages,
+    StageToAxesMap* stage_to_axes) const {
+  // Temporal object to be used if the input pointer is nullptr
+  Array<te::Stage> temp_stages;
+  StageToAxesMap temp_stage_to_axes;
+  if (stages == nullptr) {
+    stages = &temp_stages;
+  }
+  if (stage_to_axes == nullptr) {
+    stage_to_axes = &temp_stage_to_axes;
+  }
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages->push_back(stage);
+    UpdateStageAxis(stage, stage_to_axes);
+  }
+
+  // Use complete rate for the study in the paper
+  const char* complete_rate_str = getenv("ANSOR_PROGRAM_COMPLETE_RATE");
+  double complete_rate = -1.0;
+  if (complete_rate_str) {
+    complete_rate = std::stod(complete_rate_str);
+  }
+  size_t ct = 0;
+  // Apply the history steps to TVM schedule
+  for (const auto& step : transform_steps) {
+    if (complete_rate >= 0 && ct++ > transform_steps.size() * complete_rate) {
+      break;
+    }
+    // Call each step's ApplyToSchedule method
+    // Note: some steps have extra parameters that must be passed and they may need different
+    // return value, so the ApplyToSchedule is not able to be merged to single interface
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ps->ApplyToSchedule(stages, stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return std::make_pair(schedule, operator->()->tensors);
+}
+
+String ComputeDAG::PrintStepsAsPython(const Array<Step>& transform_steps) const {
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  Array<te::Operation> ops;
+  for (const auto& op : operator->()->ops) {
+    if (!op->IsInstance<te::PlaceholderOpNode>()) {
+      ops.push_back(op);
+    }
+  }
+  // Create the initial schedule
+  te::Schedule schedule = te::create_schedule({ops.back()});
+
+  // init axes
+  for (const auto& x : operator->()->ops) {
+    const te::Stage& stage = schedule.operator[](x);
+    stages.push_back(stage);
+    UpdateStageAxis(stage, &stage_to_axes);
+  }
+
+  std::stringstream ss;
+  for (const auto& stage : stages) {
+    if (stage->op->IsInstance<te::ComputeOpNode>()) {
+      for (size_t i = 0; i < stage->leaf_iter_vars.size(); ++i) {
+        ss << stage->leaf_iter_vars[i]->var->name_hint;
+        if (i != stage->leaf_iter_vars.size() - 1) {
+          ss << ", ";
+        }
+      }
+      ss << " = "
+         << "tuple(" << stage->op->name << ".op.axis)"
+         << " + "
+         << "tuple(" << stage->op->name << ".op.reduce_axis)\n";
+    }
+  }
+  // Call each step's PrintAsPythonAPI method
+  for (const auto& step : transform_steps) {
+    if (auto ps = step.as<ReorderStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<SplitStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else if (auto ps = step.as<FuseStepNode>()) {
+      ss << ps->PrintAsPythonAPI(&stages, &stage_to_axes);
+    } else {
+      LOG(FATAL) << "Invalid Step";
+    }
+  }
+
+  return ss.str();
+}
+
+State ComputeDAG::InferBound(const State& state) const {
+  State ret_state;
+  StateNode* pstate;
+
+  if (state->stages.size()) {
+    ret_state = state;
+    pstate = ret_state.CopyOnWrite();
+  } else {
+    // If the input state is incomplete with empty operation stage
+    // create a new state from init_state and update it first
+    ret_state = operator->()->init_state;
+    pstate = ret_state.CopyOnWrite();
+    pstate->transform_steps = state->transform_steps;
+    ret_state.DoSteps((*this));
+  }
+
+  Array<te::Stage> stages;
+  StageToAxesMap stage_to_axes;
+  te::Schedule sch;
+  Array<te::Tensor> tensors;
+  // Replay steps to tvm::Schedule
+  std::tie(sch, tensors) = ApplySteps(pstate->transform_steps, &stages, &stage_to_axes);
+  sch = sch.normalize();
+  // Get bound information from TVM schedule
+  Map<IterVar, Range> bounds = te::InferBound(sch);
+
+  // Update the state bound information
+  for (size_t i = 0; i < pstate->stages.size(); ++i) {
+    const Stage& stage = pstate->stages[i];
+
+    if (stage->compute_at == kInlined) {
+      continue;
+    }
+
+    Array<Iterator> new_iters;
+    new_iters.reserve(stage->iters.size());
+    // Get bound information from schedule
+    // the StageToAxesMap is used to find the corresponding IterVar in TVM schedule result
+    for (size_t j = 0; j < stage->iters.size(); ++j) {
+      const Iterator& iter = stage->iters[j];
+      const IterVar& axis = stage_to_axes.at(stages[i])[j];
+
+      auto find_res = bounds.find(axis);
+      if (find_res != bounds.end()) {
+        new_iters.push_back(
+            Iterator(iter->name, (*find_res).second, iter->iter_type, iter->annotation));
+      } else {
+        LOG(FATAL) << "Infer bound fails";
+      }
+    }
+
+    pstate->stages.Set(
+        i, Stage(stage->op, stage->op_type, std::move(new_iters), stage->compute_at, stage->attrs));
+  }
+
+  return ret_state;
+}
+
+void ComputeDAG::InferBound(Array<State>* states) const {

Review comment:
       It's fine to just return the new states:
   
   ```suggestion
   Array<State> ComputeDAG::InferBound(const Array<State>& states) const {
   ```




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/auto_schedule.py
##########
@@ -0,0 +1,186 @@
+# 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.
+
+"""User interface for auto-scheduler"""
+
+import tvm._ffi
+from tvm.runtime import Object
+from .measure import LocalBuilder, LocalRunner
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.HardwareParams")
+class HardwareParams(Object):
+    """ The parameters of target hardware, this is used to guide the search process of
+    SearchPolicy.
+
+    Parameters
+    ----------
+    num_cores : int
+        The number of device cores.
+    vector_unit_bytes : int
+        The width of vector units in bytes.
+    cache_line_bytes : int
+        The size of cache line in bytes.
+    max_unroll_vec : int
+        The max length of an axis to be unrolled or vectorized.
+    max_innermost_split_factor : int

Review comment:
       TODO added above.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: src/auto_schedule/utils.cc
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file auto_schedule/utils.cc
+ * \brief Common utilities.
+ */
+
+#include "utils.h"
+
+namespace tvm {
+namespace auto_schedule {
+
+NullStream& NullStream::Global() {
+  static NullStream stream;
+  return stream;
+}
+
+ThreadPool& ThreadPool::Global() {
+  static ThreadPool* pool = new ThreadPool();
+  static int ct = 0;
+
+  ct = (ct + 1) % ThreadPool::REFRESH_EVERY;
+
+  if (ct == 0) {
+    pool->Abort();
+    delete pool;
+    pool = new ThreadPool();
+  }
+
+  if (pool->NumWorkers() == 0) {
+    pool->Launch(std::thread::hardware_concurrency());
+  }
+
+  return *pool;
+}
+
+void parallel_for(int start, int end, std::function<void(int index)> f, int stride) {

Review comment:
       @tqchen Ok, I understand that(the stride argument has been set to 1 in default in utils.h), and it's fine to further clean these code.
   Just confused about the "does not have to change now" above. :)




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/utils.py
##########
@@ -0,0 +1,195 @@
+# 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.
+
+""" Common utilities for ansor. """
+
+from typing import Hashable
+import multiprocessing
+import multiprocessing.pool
+import queue
+import signal
+
+try:
+    import psutil
+except ImportError:
+    raise ImportError("psutil not found, try `pip install psutil` to fix this")
+
+from tvm.tir import expr
+from tvm.tir.transform import Simplify
+from tvm.ir.transform import Sequential
+from ..te import Tensor, placeholder
+
+
+def get_func_name(func):
+    """Get name of a function.
+
+    Parameters
+    ----------
+    func: Function
+        The input function.
+
+    Returns
+    -------
+    name: str
+        The function name.
+    """
+    return func.func_name if hasattr(func, 'func_name') else func.__name__
+
+
+def get_const_int(exp):
+    """Verifies expr is integer and get the constant value.
+
+    Parameters
+    ----------
+    exp : Union[tvm.tir.expr, int]
+        The input expression.
+
+    Returns
+    -------
+    out_value : int
+        The output.
+    """
+    if isinstance(exp, int):
+        return exp
+    if not isinstance(exp, expr.IntImm):
+        opt = Sequential([Simplify()])
+        exp = opt(exp)
+    if not isinstance(exp, expr.IntImm):
+        raise ValueError("Expect value to be constant int")
+    return exp.value
+
+
+def get_const_tuple(in_tuple):
+    """Verifies input tuple is IntImm, returns tuple of int.
+
+    Parameters
+    ----------
+    in_tuple : Tuple[tvm.tir.expr]
+        The input.
+
+    Returns
+    -------
+    out_tuple : Tuple[int]
+        The output.
+    """
+    return tuple(get_const_int(x) for x in in_tuple)
+
+
+
+def list_to_tuple(x):
+    """ Convert a list to a tuple recursively. """
+    assert isinstance(x, list)
+    return tuple(list_to_tuple(y) if isinstance(y, list) else y for y in x)
+
+
+def serialize_args(args):
+    """
+    Serialize arguments of a function to a hashable and jsonable tuple.
+    Currently this is mainly used for tvm.tensor.Tensor
+    """
+    ret = []
+    for t in args:
+        if isinstance(t, Tensor):
+            t = ('TENSOR', get_const_tuple(t.shape), t.dtype)
+        elif isinstance(t, list):
+            t = list_to_tuple(t)
+
+        assert isinstance(t, Hashable), str(t) + " is not hashable"
+        ret.append(t)
+
+    return tuple(ret)
+
+
+def deserialize_args(args):
+    """The inverse function of :code:`serialize_args`"""
+    ret = []
+    for t in args:
+        if isinstance(t, (tuple, list)) and t[0] == 'TENSOR':
+            ret.append(placeholder(shape=t[1], dtype=t[2]))
+        else:
+            ret.append(t)
+    return ret

Review comment:
       Thanks! We does have planed to merge AutoTVM & Ansor to one system. In this PR, we're just trying to make the Ansor code base independent.




----------------------------------------------------------------
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 #5962: [Ansor][AutoTVM v2.0] Part 0: Ansor minimum system for auto schedule generating

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



##########
File path: python/tvm/ansor/compute_dag.py
##########
@@ -0,0 +1,103 @@
+# 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.
+
+""" Computational graph and its analysis tools """
+
+import tvm._ffi
+from tvm.runtime import Object
+from .loop_state import State, StateObject
+from . import _ffi_api
+
+
+@tvm._ffi.register_object("ansor.ComputeDAG")
+class ComputeDAG(Object):
+    """
+    Computation declaration graph.
+
+    Parameters
+    ----------
+    tensors : List[Tensor]
+        `Tensor`s for a compute declaration.
+    """
+    def __init__(self, tensors):
+        self.__init_handle_by_constructor__(_ffi_api.ComputeDAG, tensors)
+
+    def get_init_state(self):

Review comment:
       Update the implementation of ComputeDAG to have a complete State class member, this function now looks better.




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