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/01/07 18:57:42 UTC

[GitHub] [incubator-tvm] icemelon9 opened a new pull request #4644: [Draft] Relay op strategy

icemelon9 opened a new pull request #4644: [Draft] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644
 
 
   Thanks for contributing to TVM!   Please refer to guideline https://docs.tvm.ai/contribute/ for useful information and tips. After the pull request is submitted, please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @ them in the pull request thread.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r374619441
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
 
 Review comment:
   why is this ok? suppose there is two Any in the code, do they get mapped to the same symbolic shape?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 edited a comment on issue #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 edited a comment on issue #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-586483826
 
 
   ci is green now. 
   Could you help review the PR? cc @tqchen @jwfromm @zhiics @ZihengJiang @junrushao1994 @merrymercy @vinx13 @kevinthesun @comaniac @tmoreau89 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r377403933
 
 

 ##########
 File path: include/tvm/te/schedule.h
 ##########
 @@ -742,6 +743,55 @@ class SingletonNode : public IterVarRelationNode {
   TVM_DECLARE_FINAL_OBJECT_INFO(SingletonNode, IterVarRelationNode);
 };
 
+class SpecializedConditionNode;
+
+/*!
+ * \brief Specialized condition to enable op specialization
+ */
+class SpecializedCondition : public ObjectRef {
+ public:
+  SpecializedCondition() {}
+  explicit SpecializedCondition(ObjectPtr<Object> n) : ObjectRef(n) {}
+  /*!
+   * \brief Get the current specialized condition.
+   * \return The current specialized condition.
+   */
+  TVM_DLL static SpecializedCondition Current();
+
+  const SpecializedConditionNode* operator->() const;
+
+  using ContainerType = SpecializedConditionNode;
+  class Internal;
+ private:
+  // enable with syntax.
+  friend class Internal;
+  friend class With<SpecializedCondition>;
+  /*! \brief Push a new specialized condition onto the thread local stack. */
+  TVM_DLL void EnterWithScope();
+  /*! \brief Pop a specialized condition off the thread local context stack. */
+  TVM_DLL void ExitWithScope();
+};
+
+/*! \brief Container for specialization conditions. */
+class SpecializedConditionNode : public Object {
+ public:
+  /*!
+   * \brief List of conditions in conjunctive joint form (CNF).
+   *   Each condition should be a simple expression, e.g., n > 16, m % 8 == 0, etc.,
+   *   where n, m are tvm::Var that represents a dimension in the tensor shape.
+   */
+  Array<PrimExpr> clauses;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("clauses", &clauses);
+  }
+
+  static SpecializedCondition make(Array<PrimExpr> conditions);
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379988301
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,190 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
 
 Review comment:
   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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376621668
 
 

 ##########
 File path: python/tvm/autotvm/task/space.py
 ##########
 @@ -944,11 +939,10 @@ def from_json_dict(json_dict):
                 raise RuntimeError("Invalid config knob type: " + knob_type)
             entity_map[str(key)] = entity
 
-        return ConfigEntity(index, code_hash, template_key, entity_map, constraints)
+        return ConfigEntity(index, code_hash, entity_map, constraints)
 
     def __repr__(self):
-        return "%s,%s,%s,%d" % (str(self._entity_map)[12:-1], self.template_key,
-                                self.code_hash, self.index)
+        return "%s,%s,%d" % (str(self._entity_map)[12:-1], self.code_hash, self.index)
 
 Review comment:
   Yes. Anyway this PR changes the workload name. So old tuning logs can no longer be 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on issue #4644: [Draft] Relay op strategy

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on issue #4644: [Draft] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-577928609
 
 
   When is conv2d gonna be here? Need it for training.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380341368
 
 

 ##########
 File path: python/tvm/autotvm/database.py
 ##########
 @@ -167,6 +167,7 @@ def filter(self, func):
             current = self.get(key)
             try:
                 records = [decode(x) for x in current.split(RedisDatabase.MAGIC_SPLIT)]
+                records = list(filter(None, records))
 
 Review comment:
   Better to use the similar logic as above and avoid `filter`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r375569255
 
 

 ##########
 File path: include/tvm/te/schedule.h
 ##########
 @@ -742,6 +743,55 @@ class SingletonNode : public IterVarRelationNode {
   TVM_DECLARE_FINAL_OBJECT_INFO(SingletonNode, IterVarRelationNode);
 };
 
+class SpecializedConditionNode;
+
+/*!
+ * \brief Specialized condition to enable op specialization
+ */
+class SpecializedCondition : public ObjectRef {
+ public:
+  SpecializedCondition() {}
+  explicit SpecializedCondition(ObjectPtr<Object> n) : ObjectRef(n) {}
+  /*!
+   * \brief Get the current specialized condition.
+   * \return The current specialized condition.
+   */
+  TVM_DLL static SpecializedCondition Current();
+
+  const SpecializedConditionNode* operator->() const;
+
+  using ContainerType = SpecializedConditionNode;
+  class Internal;
+ private:
+  // enable with syntax.
+  friend class Internal;
+  friend class With<SpecializedCondition>;
+  /*! \brief Push a new specialized condition onto the thread local stack. */
+  TVM_DLL void EnterWithScope();
+  /*! \brief Pop a specialized condition off the thread local context stack. */
+  TVM_DLL void ExitWithScope();
+};
+
+/*! \brief Container for specialization conditions. */
+class SpecializedConditionNode : public Object {
+ public:
+  /*!
+   * \brief List of conditions in conjunctive joint form (CNF).
+   *   Each condition should be a simple expression, e.g., n > 16, m % 8 == 0, etc.,
+   *   where n, m are tvm::Var that represents a dimension in the tensor shape.
+   */
+  Array<PrimExpr> clauses;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("clauses", &clauses);
+  }
+
+  static SpecializedCondition make(Array<PrimExpr> conditions);
 
 Review comment:
   probably no make, but just directly create a constructor in SpecializedCondition as this is more widely used in the new object protocol?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875266
 
 

 ##########
 File path: python/tvm/relay/op/op.py
 ##########
 @@ -143,39 +144,208 @@ class OpPattern(object):
     OPAQUE = 8
 
 
-def register_schedule(op_name, schedule=None, level=10):
-    """Register schedule function for an op
+@tvm._ffi.register_object("relay.OpImplement")
+class OpImplement(Object):
+    """Operator implementation"""
+    def compute(self, attrs, inputs, out_type):
+        """Call compute function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        inputs : list[tvm.Tensor]
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379987954
 
 

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +181,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE = {}
+
+class TopiTemplate(object):
+    """Topi template that holds the topi compute and schedule function"""
+    def __init__(self):
+        self.compute = None
+        self.schedule = None
+        self.customized_func = None
+
+    def __call__(self, *args, **kwargs):
+        args = deserialize_args(args)
+        if self.customized_func is None:
+            return self._default_func(*args, **kwargs)
+        assert callable(self.customized_func)
+        return self.customized_func(*args, **kwargs)
+
+    def _default_func(self, *args, **kwargs):
+        assert callable(self.compute) and callable(self.schedule)
+        out = self.compute(*args, **kwargs)
+        arg_bufs = [out] + self.get_inputs(out)
+        s = self.schedule([out])
+        return s, arg_bufs
+
+    def get_inputs(self, out):
+        inputs = []
+        queue = [out]
+        while queue:
+            t = queue.pop(0)
+            if isinstance(t.op, tensor.PlaceholderOp):
+                inputs.append(t)
+            else:
+                queue.extend(t.op.input_tensors)
+        return inputs
 
-def register(name, func=None, override=False):
-    """Register a task function.
+def register_task_compute(name, func=None):
+    """Register compute function to autotvm task
 
     Parameters
     ----------
-    name : str
-        The name to identify the task.
-    func : callable
-        The function to be registered.
-    override : bool
-        Whether override existing registration.
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
 
     Returns
     -------
-    func: callable
-        The registered function
+    decorator: callable
+        A decorator
     """
-    def _do_reg(myf):
-        if name in TASK_TABLE and not override:
-            raise ValueError(
-                "Key %s is already registered" % name)
-        TASK_TABLE[name] = myf
-        return myf
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.compute is not None:
+            raise ValueError("Compute is already registered in autoTVM task %s" % name)
+        tmpl.compute = f
+        return f
     if func:
         return _do_reg(func)
     return _do_reg
 
-def create(func_name, args, target, target_host=None, template_key=None):
+def register_task_schedule(name, func=None):
+    """Register schedule function to autotvm task
+
 
 Review comment:
   Added and update the 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380386961
 
 

 ##########
 File path: python/tvm/autotvm/database.py
 ##########
 @@ -125,7 +125,7 @@ def load(self, inp, get_all=False):
         current = self.get(measure_str_key(inp))
         if current is not None:
             records = [decode(x) for x in current.split(RedisDatabase.MAGIC_SPLIT)]
-            results = [rec[1] for rec in records]
+            results = [rec[1] for rec in records if rec is not None]
             if get_all:
                 return results
             return max(results, key=lambda result: result.timestamp)
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373139677
 
 

 ##########
 File path: python/tvm/autotvm/task/dispatcher.py
 ##########
 @@ -481,8 +412,12 @@ def _query_inside(self, target, workload):
         """
         if self._counter < len(self._records):
             cfg = self._records[self._counter][0].config
+            wkl = self._records[self._counter][0].task.workload
+            if workload is not None:
+                assert wkl == workload
             self._counter += 1
-            self.update(target, workload, cfg)
+            self.update(target, wkl, cfg)
+            cfg.workload = wkl
 
 Review comment:
   This is only specific to `ApplyGraphBest`. The reason is complication. Because `ApplyGraphBest` relies on the order of query, we cannot use `relay.backend.compile_engine.select_implement` to collect the autotvm workload as it may query more than once. Therefore, this is a temporary work around that we sneak in the workload in the return cfg. We can remove this part of logic after we make `ApplyGraphBest` no longer relies on the query order.
   @kevinthesun 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-580955527
 
 
   cc @merrymercy @vinx13 @ZihengJiang @jwfromm please help review if you have time

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376167957
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +85,317 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implement from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
+        The best op implementation and the corresponding output tensors.
+    """
+    all_impls = get_valid_implements(op, attrs, inputs, out_type, target)
+
+    best_plevel_impl = None
+    for impl in all_impls:
+        if best_plevel_impl is None or impl.plevel > best_plevel_impl.plevel:
+            best_plevel_impl = impl
+    if not use_autotvm:
+        outs = best_plevel_impl.compute(attrs, inputs, out_type)
+        return best_plevel_impl, outs
+
+    outputs = {}
+    best_autotvm_impl = None
+    best_cfg = None
+    dispatch_ctx = autotvm.task.DispatchContext.current
+    for impl in all_impls:
+        outs = impl.compute(attrs, inputs, out_type)
+        outputs[impl] = outs
+        workload = autotvm.task.get_workload(outs)
+        if workload is None:
+            continue
+        cfg = dispatch_ctx.query(target, workload)
+        if cfg.cost is None:
+            # It's a fallback config
+            continue
+        if best_cfg is None or best_cfg.cost > cfg.cost:
+            best_autotvm_impl = impl
+            best_cfg = cfg
+    if best_autotvm_impl:
+        return best_autotvm_impl, outputs[best_autotvm_impl]
+    return best_plevel_impl, outputs[best_plevel_impl]
+
+
+class ScheduleGetter(ExprVisitor):
 
 Review comment:
   So we will overwrite/hide the C++ side ScheduleGetter, right? I understand we need to have some opstrategy handling here, but should we reuse some code from c++ and pass through PackedFunc here in the future? Maybe not quite necessary because we currently are not really able use pure C++ env as many schedules are written 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875256
 
 

 ##########
 File path: python/tvm/autotvm/task/topi_integration.py
 ##########
 @@ -76,250 +40,49 @@ class TaskExtractEnv:
     registered = None
 
     def __init__(self, allow_duplicate=False):
-        # pylint: disable=import-outside-toplevel
-        import topi
-
-        # topi compute -> autotvm task name
-        self.topi_to_task = {
-            topi.nn.conv2d: "topi_nn_conv2d",
-            topi.nn.depthwise_conv2d_nchw: "topi_nn_depthwise_conv2d_nchw",
-            topi.nn.group_conv2d_nchw: "topi_nn_group_conv2d_nchw",
-            topi.nn.conv2d_transpose_nchw: "topi_nn_conv2d_transpose_nchw",
-            topi.nn.conv2d_NCHWc: "topi_x86_conv2d_NCHWc",
-            topi.nn.conv2d_NCHWc_int8: "topi_x86_conv2d_NCHWc_int8",
-            topi.nn.dense: "topi_nn_dense",
-            topi.nn.batch_matmul: "topi_nn_batch_matmul",
-            topi.nn.bitserial_conv2d_nchw: "topi_nn_bitserial_conv2d_nchw",
-            topi.nn.bitserial_conv2d_nhwc: "topi_nn_bitserial_conv2d_nhwc",
-            topi.nn.bitserial_dense: "topi_nn_bitserial_dense",
-            topi.nn.deformable_conv2d_nchw: "topi_nn_deformable_conv2d_nchw",
-            topi.nn.conv1d_transpose_ncw: "topi_nn_conv1d_transpose_ncw",
-            topi.nn.conv3d: "topi_nn_conv3d",
-        }
-
-        self.topi_to_schedule = {
-            topi.nn.conv2d: [topi.generic.schedule_conv2d_nchw,
-                             topi.generic.schedule_conv2d_nhwc],
-            topi.nn.depthwise_conv2d_nchw: [topi.generic.schedule_depthwise_conv2d_nchw,
-                                            topi.generic.schedule_depthwise_conv2d_nhwc],
-            topi.nn.group_conv2d_nchw: [topi.generic.schedule_group_conv2d_nchw],
-            topi.nn.conv2d_transpose_nchw: [topi.generic.schedule_conv2d_transpose_nchw],
-            topi.nn.conv2d_NCHWc: [topi.generic.schedule_conv2d_NCHWc],
-            topi.nn.conv2d_NCHWc_int8: [topi.generic.schedule_conv2d_NCHWc_int8],
-            topi.nn.dense: [topi.generic.schedule_dense],
-            topi.nn.batch_matmul: [topi.generic.schedule_batch_matmul],
-            topi.nn.bitserial_conv2d_nchw: [topi.generic.schedule_bitserial_conv2d_nchw],
-            topi.nn.bitserial_conv2d_nhwc: [topi.generic.schedule_bitserial_conv2d_nhwc],
-            topi.nn.bitserial_dense: [topi.generic.schedule_bitserial_dense],
-            topi.nn.deformable_conv2d_nchw: [topi.generic.schedule_deformable_conv2d_nchw],
-            topi.nn.conv1d_transpose_ncw: [topi.generic.schedule_conv1d_transpose_ncw],
-            topi.nn.conv3d: [topi.generic.schedule_conv3d_ndhwc],
-        }
-
-        # function reflection for tracing
-        self.func_to_reflection = {
-            topi.nn.conv2d:                 lambda x: setattr(topi.nn, 'conv2d', x),
-            topi.nn.conv2d_NCHWc:           lambda x: setattr(topi.nn, 'conv2d_NCHWc', x),
-            topi.nn.conv2d_NCHWc_int8:      lambda x: setattr(topi.nn, 'conv2d_NCHWc_int8', x),
-            topi.nn.depthwise_conv2d_nchw:  lambda x: setattr(topi.nn, 'depthwise_conv2d_nchw', x),
-            topi.nn.group_conv2d_nchw:      lambda x: setattr(topi.nn, 'group_conv2d_nchw', x),
-            topi.nn.conv2d_transpose_nchw:  lambda x: setattr(topi.nn, 'conv2d_transpose_nchw', x),
-            topi.nn.dense:                  lambda x: setattr(topi.nn, 'dense', x),
-            topi.nn.batch_matmul:           lambda x: setattr(topi.nn, 'batch_matmul', x),
-            topi.nn.bitserial_conv2d_nchw:  lambda x: setattr(topi.nn, 'bitserial_conv2d_nchw', x),
-            topi.nn.bitserial_conv2d_nhwc:  lambda x: setattr(topi.nn, 'bitserial_conv2d_nhwc', x),
-            topi.nn.bitserial_dense:        lambda x: setattr(topi.nn, 'bitserial_dense', x),
-            topi.nn.deformable_conv2d_nchw: lambda x: setattr(topi.nn, 'deformable_conv2d_nchw', x),
-            topi.nn.conv1d_transpose_ncw:   lambda x: setattr(topi.nn, 'conv1d_transpose_ncw', x),
-            topi.nn.conv3d:                 lambda x: setattr(topi.nn, 'conv3d', x),
-        }
-
         self.allow_duplicate = allow_duplicate
-        self._register_topi_task()
         self.task_collection = []
-        self.wanted_topi_funcs = list(self.topi_to_task.keys())
+        self.wanted_relay_ops = None
         self.modified_funcs = []
+        self.tracing = False
 
     def __enter__(self):
         self.task_collection = []
-        self.modified_funcs = []
-
-        for topi_compute in self.wanted_topi_funcs:
-            def _local_scope(compute_func):
-                """start a scope to hold the local function in for loop"""
-
-                def _tracing_wrapper(*args, **kwargs):
-                    assert not kwargs, "Do not support extracting tuning tasks when " \
-                                       "kwargs is used in TOPI function call. " \
-                                       "Please modify it to use only positional args."
-                    key = (self.topi_to_task[compute_func], serialize_args(args))
-                    if self.allow_duplicate or key not in self.task_collection:
-                        self.task_collection.append(key)
-
-                    return compute_func(*args, **kwargs)
-
-                self.func_to_reflection[compute_func](_tracing_wrapper)
-                self.modified_funcs.append(compute_func)
-
-            _local_scope(topi_compute)
+        self.tracing = True
 
         return self
 
     def __exit__(self, exc_type, exc_val, exc_tb):
-        # revert modification
-        for func in self.modified_funcs:
-            self.func_to_reflection[func](func)
-
-    def _register_topi_task(self):
-        """register tuning wrapper for topi function"""
-        # pylint: disable=import-outside-toplevel
-        import topi
-
-        # Avoid double registration for certain targets
-        if TaskExtractEnv.registered:
-            return
-        TaskExtractEnv.registered = True
-
-        # Tuning wrapper for topi functions
-        @register("topi_nn_conv2d")
-        def _topi_nn_conv2d(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            layout = args[-2]
-            C = topi.nn.conv2d(*args, **kwargs)
-            if layout == 'NCHW':
-                s = topi.generic.schedule_conv2d_nchw([C])
-            elif layout == 'HWCN':
-                s = topi.generic.schedule_conv2d_hwcn([C])
-            elif layout == 'NHWC':
-                s = topi.generic.schedule_conv2d_nhwc([C])
-            else:
-                raise ValueError("Unsupported layout {}".format(layout))
-            return s, [A, W, C]
-
-        @register("topi_nn_depthwise_conv2d_nchw")
-        def _topi_nn_depthwise_conv2d_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.depthwise_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.schedule_depthwise_conv2d_nchw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_group_conv2d_nchw")
-        def _topi_nn_group_conv2d_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.group_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.schedule_group_conv2d_nchw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_conv2d_transpose_nchw")
-        def _topi_nn_conv2d_transpose_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv2d_transpose_nchw(*args, **kwargs)
-            s = topi.generic.schedule_conv2d_transpose_nchw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_conv1d_transpose_ncw")
-        def _topi_nn_conv1d_transpose_ncw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv1d_transpose_ncw(*args, **kwargs)
-            s = topi.generic.schedule_conv1d_transpose_ncw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_conv3d")
-        def _topi_nn_conv3d(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv3d(*args, **kwargs)
-            s = topi.generic.schedule_conv3d_ndhwc([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_dense")
-        def _topi_nn_dense(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            if len(args) > 2:
-                data, weight, bias = args[:3]
-            else:
-                data, weight = args
-                bias = None
-            C = topi.nn.dense(*args, **kwargs)
-            s = topi.generic.schedule_dense([C])
-            if bias is not None:
-                return s, [data, weight, bias, C]
-            return s, [data, weight, C]
-
-        @register("topi_nn_batch_matmul")
-        def _topi_nn_batch_matmul(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, B = args
-            C = topi.nn.batch_matmul(A, B)
-            s = topi.generic.schedule_batch_matmul([C])
-            return s, [A, B, C]
-
-        @register("topi_nn_bitserial_conv2d_nhwc")
-        def _topi_bitserial_conv2d_nhwc(*args, **kwargs):
-            args = deserialize_args(args)
-            C = topi.nn.bitserial_conv2d_nhwc(*args, **kwargs)
-            s = topi.generic.nn.schedule_bitserial_conv2d_nhwc([C])
-            A, W = args[:2]
-            return s, [A, W, C]
-
-        @register("topi_nn_bitserial_conv2d_nchw")
-        def _topi_bitserial_conv2d_nchw(*args, **kwargs):
-            args = deserialize_args(args)
-            C = topi.nn.bitserial_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.nn.schedule_bitserial_conv2d_nchw([C])
-            A, W = args[:2]
-            return s, [A, W, C]
-
-        @register("topi_nn_bitserial_dense")
-        def _topi_nn_bitserial_dense(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.bitserial_dense(*args, **kwargs)
-            s = topi.generic.schedule_bitserial_dense([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_deformable_conv2d_nchw")
-        def _topi_nn_deformable_conv2d_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, Offset, W = args[:3]
-            C = topi.nn.deformable_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.schedule_deformable_conv2d_nchw([C])
-            return s, [A, Offset, W, C]
-
-        @register("topi_nn_conv2d_NCHWc")
-        def _topi_nn_conv2d_NCHWc(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv2d_NCHWc(*args, **kwargs)
-            s = topi.generic.schedule_conv2d_NCHWc([C])
-            return s, [A, W, C]
+        self.tracing = False
 
-    def reset(self, wanted_topi_funcs):
+    def reset(self, wanted_relay_ops=None):
         """Reset task collections
 
         Parameters
         ----------
-        wanted_topi_funcs: List of function
-            The topi function to be extracted
+        wanted_relay_ops: List of relay.op.Op
+            The relay ops to be extracted
         """
         self.task_collection = []
-        self.wanted_topi_funcs = wanted_topi_funcs
+        self.wanted_relay_ops = wanted_relay_ops
+
+    def add_task(self, task_name, args):
+        """Add AutoTVM task
+
+        Parameters
+        ----------
+        task_name: str
+            AutoTVM task name.
+
+        args: tuple
+            Arguments to the TOPI function.
+
+        cond: SpecializedCondition
 
 Review comment:
   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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380015405
 
 

 ##########
 File path: src/relay/op/nn/convolution.h
 ##########
 @@ -153,6 +153,16 @@ bool Conv2DRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
       << " But got " << out_layout;
 
   Array<IndexExpr> dshape_nchw = trans_in_layout.ForwardShape(data->shape);
+  bool is_depthwise = false;
+  if (param->groups > 1) {
+    CHECK(weight && weight->shape.defined()) <<
+        "Weight shape must be specified when groups is greater than 1.";
+    Array<IndexExpr> wshape_oihw = trans_kernel_layout.ForwardShape(weight->shape);
+    if (tvm::tir::Equal(param->groups, dshape_nchw[1]) &&
+        tvm::tir::Equal(param->groups, wshape_oihw[0])) {
 
 Review comment:
   shouldn't it be `wshape_oihw[1]`? To me it looks like testing groups == in_channel == out_channel

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379988284
 
 

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +181,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE = {}
+
+class TopiTemplate(object):
+    """Topi template that holds the topi compute and schedule function"""
+    def __init__(self):
+        self.compute = None
+        self.schedule = None
+        self.customized_func = None
+
+    def __call__(self, *args, **kwargs):
+        args = deserialize_args(args)
+        if self.customized_func is None:
+            return self._default_func(*args, **kwargs)
+        assert callable(self.customized_func)
+        return self.customized_func(*args, **kwargs)
+
+    def _default_func(self, *args, **kwargs):
+        assert callable(self.compute) and callable(self.schedule)
+        out = self.compute(*args, **kwargs)
+        arg_bufs = [out] + self.get_inputs(out)
+        s = self.schedule([out])
+        return s, arg_bufs
+
+    def get_inputs(self, out):
+        inputs = []
+        queue = [out]
+        while queue:
+            t = queue.pop(0)
+            if isinstance(t.op, tensor.PlaceholderOp):
+                inputs.append(t)
+            else:
+                queue.extend(t.op.input_tensors)
+        return inputs
 
-def register(name, func=None, override=False):
-    """Register a task function.
+def register_task_compute(name, func=None):
+    """Register compute function to autotvm task
 
     Parameters
     ----------
-    name : str
-        The name to identify the task.
-    func : callable
-        The function to be registered.
-    override : bool
-        Whether override existing registration.
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
 
     Returns
     -------
-    func: callable
-        The registered function
+    decorator: callable
+        A decorator
     """
-    def _do_reg(myf):
-        if name in TASK_TABLE and not override:
-            raise ValueError(
-                "Key %s is already registered" % name)
-        TASK_TABLE[name] = myf
-        return myf
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.compute is not None:
+            raise ValueError("Compute is already registered in autoTVM task %s" % name)
+        tmpl.compute = f
+        return f
     if func:
         return _do_reg(func)
     return _do_reg
 
-def create(func_name, args, target, target_host=None, template_key=None):
+def register_task_schedule(name, func=None):
+    """Register schedule function to autotvm task
+
+    Parameters
+    ----------
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
+
+    Returns
+    -------
+    decorator: callable
+        A decorator
+    """
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.schedule is not None:
+            raise ValueError("Schedule is already registered in autoTVM task %s" % name)
+        tmpl.schedule = f
+        return f
+    if func:
+        return _do_reg(func)
+    return _do_reg
+
+def register_customized_task(name, func=None):
+    """Register a customized function to autotvm task.
+
+    Parameters
+    ----------
+    name: str
+        The task name
+
+    func: None or callable
 
 Review comment:
   Added an example and update the 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r374903457
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
 
 Review comment:
   ooh, I forgot it is by pointer...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/relay/expr.py
 ##########
 @@ -277,6 +277,12 @@ def set_params(self, params):
 
         return _expr.FunctionSetParams(self, params)
 
+    def is_primitive(self):
+        return int(self.get_attribute("Primitive")) == 1
 
 Review comment:
   Move this as a auxiliary function instead of member for now. Because member function need to be relatively stable, and we might have new attributes 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372640446
 
 

 ##########
 File path: python/tvm/build_module.py
 ##########
 @@ -425,6 +425,8 @@ def lower(sch,
         stmt = ir_pass.InstrumentBoundCheckers(stmt)
     if simple_mode:
         return stmt
+    # print('='*80)
+    # print(stmt)
 
 Review comment:
   Remove

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380386912
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
 
 Review comment:
   Renamed to OpImplementation, and also updated the related 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373142619
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
 
 Review comment:
   added the 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] Laurawly commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
Laurawly commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372704546
 
 

 ##########
 File path: topi/python/topi/intel_graphics/conv2d.py
 ##########
 @@ -443,8 +354,32 @@ def _schedule_cl_spatialpack_NCHWc(cfg, s, op):
             s[output].compute_inline()
             conv = s.outputs[0]
         SCHEDULE_OUTPUT = False
+    else: # conv2d_NCHWc_unpack
+        conv = op.input_tensors[0]
+        temp = s[conv].op.input_tensors[0]
+        kernel = s[conv].op.input_tensors[1]
+        temp_W = s.cache_read(temp, "warp", [conv])
+        conv_L = s.cache_write(conv, "local")
+        SCHEDULE_OUTPUT = True
     kernel_L = s.cache_read(kernel, "local", [conv_L])
 
+    if temp.name == "pad_temp":
+        data = temp.op.input_tensors[0]
+        # TODO(@Laurawly): Do we need to schedule pad op here?
 
 Review comment:
   Yeah we need to schedule temp.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r381065398
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implementations(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.target.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
 
 Review comment:
   OpImplementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] yzhliu commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
yzhliu commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380008720
 
 

 ##########
 File path: topi/python/topi/arm_cpu/__init__.py
 ##########
 @@ -14,13 +14,14 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
+# pylint: disable=wildcard-import
 """Schedule for ARM CPU"""
 
-from . import conv2d
-from . import depthwise_conv2d
-from . import conv2d_transpose
-from . import conv2d_int8
-from . import bitserial_conv2d
-from . import bitserial_dense
-from . import injective
+from .conv2d import *
+from .depthwise_conv2d import *
+from .conv2d_transpose import *
+from .conv2d_int8 import *
+from . import conv2d_alter_op
+from .bitserial_conv2d import *
+from .bitserial_dense import *
+from .injective import *
 
 Review comment:
   do we need this change?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-590517026
 
 
   @merrymercy Now the tophub version number has been updated and VTA TSIM test time is also back to normal after the fix. Could you review the PR again?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372636908
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
 
 Review comment:
   Could you add comments to this statement, or use a better name for `flag`?  I don't quite understand the logic 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875268
 
 

 ##########
 File path: python/tvm/relay/op/op.py
 ##########
 @@ -143,39 +144,208 @@ class OpPattern(object):
     OPAQUE = 8
 
 
-def register_schedule(op_name, schedule=None, level=10):
-    """Register schedule function for an op
+@tvm._ffi.register_object("relay.OpImplement")
+class OpImplement(Object):
+    """Operator implementation"""
+    def compute(self, attrs, inputs, out_type):
+        """Call compute function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        inputs : list[tvm.Tensor]
+            The input tensors.
+
+        out_type : relay.Type
+            The output type.
+
+        Returns
+        -------
+        outs : list[tvm.Tensor]
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r377403661
 
 

 ##########
 File path: python/tvm/schedule.py
 ##########
 @@ -650,4 +650,38 @@ def opengl(self):
         """
         _api_internal._StageOpenGL(self)
 
+@tvm._ffi.register_object
+class SpecializedCondition(Object):
+    """Specialized condition to enable op specialization."""
+    def __init__(self, conditions):
+        """Create a specialized condition.
+
+        .. note::
+            Conditions are represented in conjunctive joint form (CNF).
+            Each condition should be a simple expression, e.g., n > 16,
+            m % 8 == 0, etc., where n, m are tvm.Var that represents a
+            dimension in the tensor shape.
+
+        Parameters
+        ----------
+        conditions : List of tvm.Expr
+            List of conditions in conjunctive joint form (CNF).
+        """
+        if not isinstance(conditions, (list, _container.Array)):
+            conditions = [conditions]
+        self.__init_handle_by_constructor__(
+            _api_internal._CreateSpecializedCondition, conditions)
+
+    def __enter__(self):
+        _api_internal._EnterSpecializationScope(self)
+        return self
+
+    def __exit__(self, ptype, value, trace):
+        _api_internal._ExitSpecializationScope(self)
+
+
 
 Review comment:
   Fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372685610
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implement from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
+        The best op implementation and the corresponding output tensors.
+    """
+    all_impls = get_valid_implements(op, attrs, inputs, out_type, target)
+
+    best_plevel_impl = None
+    for impl in all_impls:
+        if best_plevel_impl is None or int(impl.plevel) > int(best_plevel_impl.plevel):
+            best_plevel_impl = impl
+    if not use_autotvm:
+        outs = best_plevel_impl.compute(attrs, inputs, out_type)
+        return best_plevel_impl, outs
+
+    outputs = {}
+    best_autotvm_impl = None
+    best_cfg = None
+    dispatch_ctx = autotvm.task.DispatchContext.current
+    for impl in all_impls:
+        outs = impl.compute(attrs, inputs, out_type)
+        outputs[impl] = outs
+        workload = autotvm.task.get_workload(outs)
+        if workload is None:
+            continue
+        workload = autotvm.task.args_to_workload(workload)
 
 Review comment:
   This looks a bit confusing. We got `workload` from `get_workload` and got a hashable workload by `args_to_workload`. I actually saw `args_to_worload` is being called in `get_workload`, so should it be safe to remove this line?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372683444
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implement from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
+        The best op implementation and the corresponding output tensors.
+    """
+    all_impls = get_valid_implements(op, attrs, inputs, out_type, target)
+
+    best_plevel_impl = None
+    for impl in all_impls:
+        if best_plevel_impl is None or int(impl.plevel) > int(best_plevel_impl.plevel):
 
 Review comment:
   Why we need to cast `plevel` to int 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
 
 Review comment:
   OpImplement -> OpImplementation?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379976934
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
 
 Review comment:
   I agree OpImplementation should be better. But this name is a bit too long. Do you have better idea?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r377403904
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Name of the implementation */
+  std::string name;
+  /*! \brief Priority level */
+  int plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
+ public:
+  /*!
+   * \brief Invoke the operator compute function.
+   * \param attrs The attribute of the primitive
+   * \param inputs The input tensors.
+   * \param out_type The output type information.
+   * \return The output compute description of the operator.
+   */
+  Array<te::Tensor> Compute(const Attrs& attrs,
+                            const Array<te::Tensor>& inputs,
+                            const Type& out_type);
+  /*!
+   * \brief Build the computation schedule.
+   * \param attrs The attribute of the node.
+   * \param outs The output tensors.
+   * \param target The build target.
+   * \return The computation schedule.
+   */
+  te::Schedule Schedule(const Attrs& attrs,
+                        const Array<te::Tensor>& outs,
+                        const Target& target);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(OpImplement, ObjectRef, OpImplementNode);
+};
+
+/*!
+ * \brief Specialized implementations for operators under certain conditions.
+ */
+class OpSpecializationNode : public Object {
+ public:
+  /*! \brief List of implementations. */
+  Array<OpImplement> implements;
+  /*! \brief Condition to enable the specialization.
+   *    Could be undefined to represent generic case. */
+  te::SpecializedCondition condition;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("condition", &condition);
+    v->Visit("implements", &implements);
+  }
+
+  static constexpr const char* _type_key = "relay.OpSpecialization";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpSpecializationNode, ExprNode);
+};
+
+/*!
+ * \brief Operator specialization class.
+ */
+class OpSpecialization : public ObjectRef {
+ public:
+  /*!
+   * \brief Add an implementation.
+   * \param compute Compute function
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376144257
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Name of the implementation */
+  std::string name;
+  /*! \brief Priority level */
+  int plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
+ public:
+  /*!
+   * \brief Invoke the operator compute function.
+   * \param attrs The attribute of the primitive
+   * \param inputs The input tensors.
+   * \param out_type The output type information.
+   * \return The output compute description of the operator.
+   */
+  Array<te::Tensor> Compute(const Attrs& attrs,
+                            const Array<te::Tensor>& inputs,
+                            const Type& out_type);
+  /*!
+   * \brief Build the computation schedule.
+   * \param attrs The attribute of the node.
+   * \param outs The output tensors.
+   * \param target The build target.
+   * \return The computation schedule.
+   */
+  te::Schedule Schedule(const Attrs& attrs,
+                        const Array<te::Tensor>& outs,
+                        const Target& target);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(OpImplement, ObjectRef, OpImplementNode);
+};
+
+/*!
+ * \brief Specialized implementations for operators under certain conditions.
+ */
+class OpSpecializationNode : public Object {
+ public:
+  /*! \brief List of implementations. */
+  Array<OpImplement> implements;
+  /*! \brief Condition to enable the specialization.
+   *    Could be undefined to represent generic case. */
+  te::SpecializedCondition condition;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("condition", &condition);
+    v->Visit("implements", &implements);
+  }
+
+  static constexpr const char* _type_key = "relay.OpSpecialization";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpSpecializationNode, ExprNode);
+};
+
+/*!
+ * \brief Operator specialization class.
+ */
+class OpSpecialization : public ObjectRef {
+ public:
+  /*!
+   * \brief Add an implementation.
+   * \param compute Compute function
+   * \param schedule Schedule function
 
 Review comment:
   s/schedule/fschedule

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373142087
 
 

 ##########
 File path: python/tvm/autotvm/task/dispatcher.py
 ##########
 @@ -481,8 +412,12 @@ def _query_inside(self, target, workload):
         """
         if self._counter < len(self._records):
             cfg = self._records[self._counter][0].config
+            wkl = self._records[self._counter][0].task.workload
+            if workload is not None:
+                assert wkl == workload
             self._counter += 1
-            self.update(target, workload, cfg)
+            self.update(target, wkl, cfg)
+            cfg.workload = wkl
 
 Review comment:
   I see. Could we define `self.workload` in `ConfigSpace` and add your comment on it? So that we will remember to remove it 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875293
 
 

 ##########
 File path: topi/python/topi/intel_graphics/conv2d.py
 ##########
 @@ -159,108 +147,59 @@ def tile_and_bind3d(s, tensor, z, y, x, z_factor=2, y_factor=None, x_factor=None
 # We define schedule template in this function instead of
 # declaration function since actual input arguments need
 # to be altered by the schedule selected.
-@autotvm.task.register("topi_intel_graphics_conv2d_NCHWc")
-def __topi_nn_conv2d_NCHWc(*args, **kwargs):
-    assert not kwargs, "Do not support kwargs in template function call"
-    data, kernel, strides, padding, dilation, layout, dtype = deserialize_args(args)
-    raw_data_shape = get_const_tuple(data.shape)
-    raw_kernel_shape = get_const_tuple(kernel.shape)
-
-    # get config here
-    cfg = get_config()
-    _create_schedule_template(cfg, data, kernel, strides, padding, dilation, layout)
-    cfg.add_flop(1)
-
-    # change shape with the value in config
-    ic_bn = cfg["tile_ic"].val if hasattr(cfg["tile_ic"], "val") else cfg["tile_ic"].size[-1]
-    oc_bn = cfg["tile_oc"].val if hasattr(cfg["tile_oc"], "val") else cfg["tile_oc"].size[-1]
-
-    new_data_shape = (raw_data_shape[0], raw_data_shape[1] // ic_bn,
-                      raw_data_shape[2], raw_data_shape[3], ic_bn)
-    new_kernel_shape = (raw_kernel_shape[0] // oc_bn, raw_kernel_shape[1] // ic_bn,
-                        raw_kernel_shape[2], raw_kernel_shape[3], ic_bn, oc_bn)
-    new_data = tvm.placeholder(new_data_shape, data.dtype)
-    new_kernel = tvm.placeholder(new_kernel_shape, kernel.dtype)
-
-    C = _decl_cl_spatialpack_NCHWc(cfg, new_data, new_kernel, strides, padding, dilation, dtype)
-    s = _schedule_conv2d_NCHWc(cfg, [C])
-
-    return s, [new_data, new_kernel, C]
-
-@conv2d_alter_layout.register(["intel_graphics"])
-def _alter_conv2d_layout(attrs, inputs, tinfo, F):
-    copy_inputs = list(inputs)
-    new_attrs = {k : attrs[k] for k in attrs.keys()}
-
-    if F.__name__ == 'tvm.relay.op':
-        # Derive channels for frontends (e.g ONNX) that miss "channel" field.
-        new_attrs["channels"] = inputs[1].checked_type.shape[attrs['kernel_layout'].index('O')]
-
-    data, kernel = tinfo[0], tinfo[1]
-    batch_size, in_channel, height, width = get_const_tuple(data.shape)
+# @autotvm.task.register("topi_intel_graphics_conv2d_NCHWc")
 
 Review comment:
   removed. these are useless 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-589723613
 
 
   Will let @merrymercy manage this PR

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373141667
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implement from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
+        The best op implementation and the corresponding output tensors.
+    """
+    all_impls = get_valid_implements(op, attrs, inputs, out_type, target)
+
+    best_plevel_impl = None
+    for impl in all_impls:
+        if best_plevel_impl is None or int(impl.plevel) > int(best_plevel_impl.plevel):
+            best_plevel_impl = impl
+    if not use_autotvm:
+        outs = best_plevel_impl.compute(attrs, inputs, out_type)
+        return best_plevel_impl, outs
+
+    outputs = {}
+    best_autotvm_impl = None
+    best_cfg = None
+    dispatch_ctx = autotvm.task.DispatchContext.current
+    for impl in all_impls:
+        outs = impl.compute(attrs, inputs, out_type)
+        outputs[impl] = outs
+        workload = autotvm.task.get_workload(outs)
+        if workload is None:
+            continue
+        workload = autotvm.task.args_to_workload(workload)
 
 Review comment:
   Yes, I forgot to remove this line. :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376162304
 
 

 ##########
 File path: include/tvm/te/schedule.h
 ##########
 @@ -742,6 +743,55 @@ class SingletonNode : public IterVarRelationNode {
   TVM_DECLARE_FINAL_OBJECT_INFO(SingletonNode, IterVarRelationNode);
 };
 
+class SpecializedConditionNode;
+
+/*!
+ * \brief Specialized condition to enable op specialization
+ */
+class SpecializedCondition : public ObjectRef {
+ public:
+  SpecializedCondition() {}
+  explicit SpecializedCondition(ObjectPtr<Object> n) : ObjectRef(n) {}
+  /*!
+   * \brief Get the current specialized condition.
+   * \return The current specialized condition.
+   */
+  TVM_DLL static SpecializedCondition Current();
+
+  const SpecializedConditionNode* operator->() const;
+
+  using ContainerType = SpecializedConditionNode;
+  class Internal;
+ private:
+  // enable with syntax.
+  friend class Internal;
+  friend class With<SpecializedCondition>;
+  /*! \brief Push a new specialized condition onto the thread local stack. */
+  TVM_DLL void EnterWithScope();
+  /*! \brief Pop a specialized condition off the thread local context stack. */
+  TVM_DLL void ExitWithScope();
+};
+
+/*! \brief Container for specialization conditions. */
+class SpecializedConditionNode : public Object {
+ public:
+  /*!
+   * \brief List of conditions in conjunctive joint form (CNF).
+   *   Each condition should be a simple expression, e.g., n > 16, m % 8 == 0, etc.,
+   *   where n, m are tvm::Var that represents a dimension in the tensor shape.
+   */
+  Array<PrimExpr> clauses;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("clauses", &clauses);
+  }
+
+  static SpecializedCondition make(Array<PrimExpr> conditions);
+
+  static constexpr const char* _type_key = "SpecializedCondition";
 
 Review comment:
   te.SpecializedCondition ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380301022
 
 

 ##########
 File path: topi/python/topi/arm_cpu/conv2d_alter_op.py
 ##########
 @@ -0,0 +1,171 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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=invalid-name,unused-variable,unused-argument,no-member
+"""Conv2D alter op and legalize functions for arm cpu"""
+
+import logging
+
+import tvm
+from tvm import relay
+from tvm import autotvm
+
+from ..nn import conv2d_alter_layout
+from ..util import get_const_tuple
+
+
+logger = logging.getLogger('topi')
+
+
+@conv2d_alter_layout.register(["arm_cpu"])
+def _alter_conv2d_layout(attrs, inputs, tinfos, out_type):
+    target = tvm.target.Target.current(allow_none=False)
+    dispatch_ctx = autotvm.task.DispatchContext.current
+
+    _, outs = relay.backend.compile_engine.select_implement(
+        relay.op.get("nn.conv2d"), attrs, tinfos, out_type, target)
+    workload = autotvm.task.get_workload(outs)
+    if workload is None:
+        # The best implementation is not an AutoTVM template,
+        # we then assume it's not necessary to alter this op.
+        return None
+    cfg = dispatch_ctx.query(target, workload)
+    if cfg.is_fallback:  # if is fallback, clear query cache and return None
+        autotvm.task.clear_fallback_cache(target, workload)
 
 Review comment:
   The dispatch context will cache the random assigned fallback cache. I think this line just clear the query cache. I just followed the original arm_cpu cov2d_layout implementation for 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376057778
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,14 +216,182 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
                      const Array<te::Tensor>& inputs,
                      const Array<IndexExpr>& out_ndims)>;
 
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Priority level */
+  Integer plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
+ public:
+  /*! \brief default constructor  */
+  OpImplement() {}
+  /*! \brief constructor from node pointer */
+  explicit OpImplement(ObjectPtr<Object> n) : ObjectRef(n) {}
+  /*!
+   * \brief access the internal node container
+   * \return the pointer to the internal node container
+   */
+  inline const OpImplementNode* operator->() const;
 
 Review comment:
   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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/relay/op/op.py
 ##########
 @@ -143,39 +144,208 @@ class OpPattern(object):
     OPAQUE = 8
 
 
-def register_schedule(op_name, schedule=None, level=10):
-    """Register schedule function for an op
+@tvm._ffi.register_object("relay.OpImplement")
+class OpImplement(Object):
+    """Operator implementation"""
+    def compute(self, attrs, inputs, out_type):
+        """Call compute function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        inputs : list[tvm.Tensor]
+            The input tensors.
+
+        out_type : relay.Type
+            The output type.
+
+        Returns
+        -------
+        outs : list[tvm.Tensor]
 
 Review comment:
   tvm.tensor.Tensor

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #4644: [Draft] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #4644: [Draft] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-579944529
 
 
   I've added the strategy for all ops. We can start to review this PR since it's huge. Could you help review it?
   @tqchen @kevinthesun @comaniac @masahi @MarisaKirisame @hlu1 @yzhliu @zhiics @ZihengJiang @merrymercy @vinx13 @FrozenGene @jroesch 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379987912
 
 

 ##########
 File path: python/tvm/relay/expr.py
 ##########
 @@ -277,6 +277,12 @@ def set_params(self, params):
 
         return _expr.FunctionSetParams(self, params)
 
+    def is_primitive(self):
+        return int(self.get_attribute("Primitive")) == 1
 
 Review comment:
   Removed this function for now as it's 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/relay/op/op.py
 ##########
 @@ -143,39 +144,208 @@ class OpPattern(object):
     OPAQUE = 8
 
 
-def register_schedule(op_name, schedule=None, level=10):
-    """Register schedule function for an op
+@tvm._ffi.register_object("relay.OpImplement")
+class OpImplement(Object):
+    """Operator implementation"""
+    def compute(self, attrs, inputs, out_type):
+        """Call compute function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        inputs : list[tvm.Tensor]
+            The input tensors.
+
+        out_type : relay.Type
+            The output type.
+
+        Returns
+        -------
+        outs : list[tvm.Tensor]
+            The output tensors.
+        """
+        return _OpImplementCompute(self, attrs, inputs, out_type)
+
+    def schedule(self, attrs, outs, target):
+        """Call schedule function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        outs : list[tvm.Tensor]
+            The output tensors.
+
+        target : tvm.Target
 
 Review comment:
   tvm.target.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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-589237682
 
 
   @yzhliu @jwfromm Please https://docs.tvm.ai/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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-588610092
 
 
   @yzhliu @zhiics @comaniac @masahi Could you review again or approve 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380386976
 
 

 ##########
 File path: python/tvm/autotvm/database.py
 ##########
 @@ -167,6 +167,7 @@ def filter(self, func):
             current = self.get(key)
             try:
                 records = [decode(x) for x in current.split(RedisDatabase.MAGIC_SPLIT)]
+                records = list(filter(None, records))
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r381029964
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    analyzer = tvm.arith.Analyzer()
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = analyzer.canonical_simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
 
 Review comment:
   changed to implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380971993
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    analyzer = tvm.arith.Analyzer()
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = analyzer.canonical_simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
 
 Review comment:
   implementation?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380386997
 
 

 ##########
 File path: python/tvm/autotvm/record.py
 ##########
 @@ -130,9 +135,17 @@ def decode(row, protocol='json'):
     result: autotvm.tuner.MeasureResult
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379874770
 
 

 ##########
 File path: python/tvm/relay/memory_alloc.py
 ##########
 @@ -28,8 +28,8 @@
 
 
 def is_primitive(call):
-    return hasattr(call.op, 'attrs') and hasattr(call.op.attrs, 'Primitive') and \
-        int(call.op.attrs.Primitive) == 1
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
 
 Review comment:
   The `call.op` could be primitive `Function` or `Op`. `Op` doesn't have `is_primitive` api.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +181,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE = {}
+
+class TopiTemplate(object):
+    """Topi template that holds the topi compute and schedule function"""
+    def __init__(self):
+        self.compute = None
+        self.schedule = None
+        self.customized_func = None
+
+    def __call__(self, *args, **kwargs):
+        args = deserialize_args(args)
+        if self.customized_func is None:
+            return self._default_func(*args, **kwargs)
+        assert callable(self.customized_func)
+        return self.customized_func(*args, **kwargs)
+
+    def _default_func(self, *args, **kwargs):
+        assert callable(self.compute) and callable(self.schedule)
+        out = self.compute(*args, **kwargs)
+        arg_bufs = [out] + self.get_inputs(out)
+        s = self.schedule([out])
+        return s, arg_bufs
+
+    def get_inputs(self, out):
+        inputs = []
+        queue = [out]
+        while queue:
+            t = queue.pop(0)
+            if isinstance(t.op, tensor.PlaceholderOp):
+                inputs.append(t)
+            else:
+                queue.extend(t.op.input_tensors)
+        return inputs
 
-def register(name, func=None, override=False):
-    """Register a task function.
+def register_task_compute(name, func=None):
+    """Register compute function to autotvm task
 
     Parameters
     ----------
-    name : str
-        The name to identify the task.
-    func : callable
-        The function to be registered.
-    override : bool
-        Whether override existing registration.
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
 
     Returns
     -------
-    func: callable
-        The registered function
+    decorator: callable
+        A decorator
     """
-    def _do_reg(myf):
-        if name in TASK_TABLE and not override:
-            raise ValueError(
-                "Key %s is already registered" % name)
-        TASK_TABLE[name] = myf
-        return myf
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.compute is not None:
+            raise ValueError("Compute is already registered in autoTVM task %s" % name)
+        tmpl.compute = f
+        return f
     if func:
         return _do_reg(func)
     return _do_reg
 
-def create(func_name, args, target, target_host=None, template_key=None):
+def register_task_schedule(name, func=None):
+    """Register schedule function to autotvm task
+
+    Parameters
+    ----------
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
+
+    Returns
+    -------
+    decorator: callable
+        A decorator
+    """
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.schedule is not None:
+            raise ValueError("Schedule is already registered in autoTVM task %s" % name)
+        tmpl.schedule = f
+        return f
+    if func:
+        return _do_reg(func)
+    return _do_reg
+
+def register_customized_task(name, func=None):
+    """Register a customized function to autotvm task.
+
+    Parameters
+    ----------
+    name: str
+        The task name
+
+    func: None or callable
 
 Review comment:
   Add an example code section here.
   
   Explain what is an customized task

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380387035
 
 

 ##########
 File path: python/tvm/autotvm/task/relay_integration.py
 ##########
 @@ -67,27 +65,22 @@ def extract_from_program(mod, params, ops, target, target_host=None,
         The module or function to tune
     params: dict of str to numpy array
         The associated parameters of the program
-    ops: List of relay op
-        List of relay ops to be tuned
     target: tvm.target.Target
         The compilation target
     target_host: tvm.target.Target
         The host compilation target
-    template_keys: dict of topi op to str
-        The tuning template keys map for schedules, default to None.
-        Example: {topi.nn.conv2d: 'direct'}
+    ops: List of relay.op.Op
+        List of relay ops to be tuned
 
     Returns
     -------
     task: Array of autotvm.task.Task
         collected tasks
     """
-    return extract_from_multiple_program([mod], [params], ops, target, target_host,
-                                         template_keys)
+    return extract_from_multiple_program([mod], [params], target, target_host, ops)
 
 
-def extract_from_multiple_program(mods, params, ops, target, target_host=None,
-                                  template_keys=None):
+def extract_from_multiple_program(mods, params, target, target_host=None, ops=None):
 
 Review comment:
   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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875269
 
 

 ##########
 File path: src/relay/op/nn/pooling.cc
 ##########
 @@ -164,9 +164,8 @@ bool Pool2DRel(const Array<Type>& types,
 
 template<typename AttrType, topi::nn::PoolType mode>
 Array<te::Tensor> Pool2DCompute(const Attrs& attrs,
-                            const Array<te::Tensor>& inputs,
-                            const Type& out_type,
-                            const Target& target) {
+                                 const Array<te::Tensor>& inputs,
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376171923
 
 

 ##########
 File path: python/tvm/schedule.py
 ##########
 @@ -650,4 +650,38 @@ def opengl(self):
         """
         _api_internal._StageOpenGL(self)
 
+@tvm._ffi.register_object
+class SpecializedCondition(Object):
+    """Specialized condition to enable op specialization."""
+    def __init__(self, conditions):
+        """Create a specialized condition.
+
+        .. note::
+            Conditions are represented in conjunctive joint form (CNF).
+            Each condition should be a simple expression, e.g., n > 16,
+            m % 8 == 0, etc., where n, m are tvm.Var that represents a
+            dimension in the tensor shape.
+
+        Parameters
+        ----------
+        conditions : List of tvm.Expr
+            List of conditions in conjunctive joint form (CNF).
+        """
+        if not isinstance(conditions, (list, _container.Array)):
+            conditions = [conditions]
+        self.__init_handle_by_constructor__(
+            _api_internal._CreateSpecializedCondition, conditions)
+
+    def __enter__(self):
+        _api_internal._EnterSpecializationScope(self)
+        return self
+
+    def __exit__(self, ptype, value, trace):
+        _api_internal._ExitSpecializationScope(self)
+
+
 
 Review comment:
   Move this into SpecializedCondition as current() and decorate it as a static method? This is more consistent to the C++ side.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] Laurawly commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
Laurawly commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372695578
 
 

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +149,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE2 = {}
 
 Review comment:
   Can we remove 2 in task_table as well?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] Laurawly commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
Laurawly commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372700311
 
 

 ##########
 File path: python/tvm/relay/quantize/_annotate.py
 ##########
 @@ -53,11 +53,11 @@ def simulated_quantize_compute(attrs, inputs, out_type, target):
     return [rdata]
 
 
-_reg.register_schedule("relay.op.annotation.simulated_quantize",
-                       _reg.schedule_injective)
+# _reg.register_schedule("relay.op.annotation.simulated_quantize",
 
 Review comment:
   Remove

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380295907
 
 

 ##########
 File path: src/relay/op/nn/convolution.h
 ##########
 @@ -153,6 +153,16 @@ bool Conv2DRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
       << " But got " << out_layout;
 
   Array<IndexExpr> dshape_nchw = trans_in_layout.ForwardShape(data->shape);
+  bool is_depthwise = false;
+  if (param->groups > 1) {
+    CHECK(weight && weight->shape.defined()) <<
+        "Weight shape must be specified when groups is greater than 1.";
+    Array<IndexExpr> wshape_oihw = trans_kernel_layout.ForwardShape(weight->shape);
+    if (tvm::tir::Equal(param->groups, dshape_nchw[1]) &&
+        tvm::tir::Equal(param->groups, wshape_oihw[0])) {
 
 Review comment:
   In depthwise conv2d, weight's out_channel == groups == input's in_channel, and weight's in_channel == channel multiplier. The output's channel = groups * channel multiplier.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] merrymercy merged pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
merrymercy merged pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r382400753
 
 

 ##########
 File path: python/tvm/autotvm/record.py
 ##########
 @@ -119,20 +124,29 @@ def decode(row, protocol='json'):
 
     Parameters
     ----------
-    row: str
+    row : str
         a row in the logger file
-    protocol: str
+
+    protocol : str
         log protocol, json or pickle
 
     Returns
     -------
-    input: autotvm.tuner.MeasureInput
-    result: autotvm.tuner.MeasureResult
+    ret : tuple(autotvm.tuner.MeasureInput, autotvm.tuner.MeasureResult), or None
+        The tuple of input and result, or None if input uses old version log format.
     """
     # pylint: disable=unused-variable
+    global _old_version_warning
+
     if protocol == 'json':
         row = json.loads(row)
-        tgt, task_name, task_args, task_kwargs, workload, config = row['i']
+        if 'v' in row and row['v'] == 0.1:
+            if _old_version_warning:
+                logger.warning("AutoTVM log version 0.1 is no longer supported.")
 
 Review comment:
   I'll do a follow up PR to tophub and try to convert to the correct log format. But I can't guarantee every record could be converted successful.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-586483826
 
 
   ci is green now. 
   Could you help review the PR? cc @tqchen @jwfromm @zhiics @ZihengJiang @junrushao1994 @merrymercy @vinx13 @kevinthesun @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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jwfromm commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
jwfromm commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r374870865
 
 

 ##########
 File path: python/tvm/relay/op/strategy/hls.py
 ##########
 @@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Definition of HLS operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from __future__ import absolute_import
+
+import topi
+from .generic import *
+from .. import op as _op
+
+@schedule_injective.register("hls")
+def schedule_injective_hls(attrs, outs, target):
+    """schedule injective ops for hls"""
+    with target:
+        return topi.hls.schedule_injective(outs)
+
+@schedule_reduce.register("hls")
+def schedule_reduce_hls(attrs, outs, target):
+    """schedule reduction ops for hls"""
+    with target:
+        return topi.hls.schedule_reduce(outs)
+
+@schedule_concatenate.register("hls")
+def schedule_concatenate_hls(attrs, outs, target):
+    """schedule concatenate for hls"""
+    with target:
+        return topi.hls.schedule_injective(outs)
+
+@schedule_pool.register("hls")
+def schedule_pool_hls(attrs, outs, target):
+    """schedule pooling ops for hls"""
+    with target:
+        return topi.hls.schedule_pool(outs, attrs.layout)
+
+@schedule_adaptive_pool.register("hls")
+def schedule_adaptive_pool_hls(attrs, outs, target):
+    """schedule adaptive pooling ops for hls"""
+    with target:
+        return topi.hls.schedule_adaptive_pool(outs)
+
+@schedule_softmax.register("hls")
+def schedule_softmax_hls(attrs, outs, target):
+    """schedule softmax for hls"""
+    with target:
+        return topi.hls.schedule_softmax(outs)
+
+@override_native_generic_func("conv2d_strategy")
+def conv2d_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d hls strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    (dilation_h, dilation_w) = dilation
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nchw),
+                wrap_topi_schedule(topi.hls.schedule_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWIO"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nhwc),
+                wrap_topi_schedule(topi.hls.schedule_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported conv2d layout {}".format(layout))
+    elif is_depthwise_conv2d(data.shape, layout, kernel.shape, kernel_layout, groups):
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nchw),
+                wrap_topi_schedule(topi.hls.schedule_depthwise_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWOI"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nhwc),
+                wrap_topi_schedule(topi.hls.schedule_depthwise_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported depthwise_conv2d layout {}".format(layout))
+    else: # group_conv2d
+        raise RuntimeError("group_conv2d is not supported for hls")
+    return strategy
+
+@override_native_generic_func("conv2d_NCHWc_strategy")
+def conv2d_NCHWc_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d_NCHWc hls strategy"""
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_compute_conv2d(topi.nn.conv2d_NCHWc, True, True),
+        wrap_topi_schedule(topi.hls.schedule_conv2d_NCHWc))
+    return strategy
+
+@conv2d_transpose_strategy.register("hls")
+def conv2d_transpose_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d_transpose hls strategy"""
+    layout = attrs.data_layout
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    assert layout == "NCHW", "only support nchw for now"
+    assert dilation == (1, 1), "not support dilate now"
+    assert groups == 1, "only support groups == 1 for now"
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_comptue_conv2d_transpose(topi.nn.conv2d_transpose_nchw),
 
 Review comment:
   compute typo in calls to `wrap_comptue_conv2d_transpose`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376057894
 
 

 ##########
 File path: python/tvm/relay/op/strategy/hls.py
 ##########
 @@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Definition of HLS operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from __future__ import absolute_import
+
+import topi
+from .generic import *
+from .. import op as _op
+
+@schedule_injective.register("hls")
+def schedule_injective_hls(attrs, outs, target):
+    """schedule injective ops for hls"""
+    with target:
+        return topi.hls.schedule_injective(outs)
+
+@schedule_reduce.register("hls")
+def schedule_reduce_hls(attrs, outs, target):
+    """schedule reduction ops for hls"""
+    with target:
+        return topi.hls.schedule_reduce(outs)
+
+@schedule_concatenate.register("hls")
+def schedule_concatenate_hls(attrs, outs, target):
+    """schedule concatenate for hls"""
+    with target:
+        return topi.hls.schedule_injective(outs)
+
+@schedule_pool.register("hls")
+def schedule_pool_hls(attrs, outs, target):
+    """schedule pooling ops for hls"""
+    with target:
+        return topi.hls.schedule_pool(outs, attrs.layout)
+
+@schedule_adaptive_pool.register("hls")
+def schedule_adaptive_pool_hls(attrs, outs, target):
+    """schedule adaptive pooling ops for hls"""
+    with target:
+        return topi.hls.schedule_adaptive_pool(outs)
+
+@schedule_softmax.register("hls")
+def schedule_softmax_hls(attrs, outs, target):
+    """schedule softmax for hls"""
+    with target:
+        return topi.hls.schedule_softmax(outs)
+
+@override_native_generic_func("conv2d_strategy")
+def conv2d_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d hls strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    (dilation_h, dilation_w) = dilation
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nchw),
+                wrap_topi_schedule(topi.hls.schedule_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWIO"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nhwc),
+                wrap_topi_schedule(topi.hls.schedule_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported conv2d layout {}".format(layout))
+    elif is_depthwise_conv2d(data.shape, layout, kernel.shape, kernel_layout, groups):
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nchw),
+                wrap_topi_schedule(topi.hls.schedule_depthwise_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWOI"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nhwc),
+                wrap_topi_schedule(topi.hls.schedule_depthwise_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported depthwise_conv2d layout {}".format(layout))
+    else: # group_conv2d
+        raise RuntimeError("group_conv2d is not supported for hls")
+    return strategy
+
+@override_native_generic_func("conv2d_NCHWc_strategy")
+def conv2d_NCHWc_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d_NCHWc hls strategy"""
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_compute_conv2d(topi.nn.conv2d_NCHWc, True, True),
+        wrap_topi_schedule(topi.hls.schedule_conv2d_NCHWc))
+    return strategy
+
+@conv2d_transpose_strategy.register("hls")
+def conv2d_transpose_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d_transpose hls strategy"""
+    layout = attrs.data_layout
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    assert layout == "NCHW", "only support nchw for now"
+    assert dilation == (1, 1), "not support dilate now"
+    assert groups == 1, "only support groups == 1 for now"
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_comptue_conv2d_transpose(topi.nn.conv2d_transpose_nchw),
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #4644: [Draft] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #4644: [Draft] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-577928887
 
 
   x86 conv2d is already in the pr. working on cuda and more targets 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380340757
 
 

 ##########
 File path: python/tvm/autotvm/database.py
 ##########
 @@ -125,7 +125,7 @@ def load(self, inp, get_all=False):
         current = self.get(measure_str_key(inp))
         if current is not None:
             records = [decode(x) for x in current.split(RedisDatabase.MAGIC_SPLIT)]
-            results = [rec[1] for rec in records]
+            results = [rec[1] for rec in records if rec is not None]
             if get_all:
                 return results
             return max(results, key=lambda result: result.timestamp)
 
 Review comment:
   `max` will throw `ValueError` if `results` is empty.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373136444
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implement from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
+        The best op implementation and the corresponding output tensors.
+    """
+    all_impls = get_valid_implements(op, attrs, inputs, out_type, target)
+
+    best_plevel_impl = None
+    for impl in all_impls:
+        if best_plevel_impl is None or int(impl.plevel) > int(best_plevel_impl.plevel):
 
 Review comment:
   Because `plevel` is `IntImm`, direct comparison between them will become an expr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jwfromm commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
jwfromm commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r374966045
 
 

 ##########
 File path: python/tvm/relay/op/strategy/x86.py
 ##########
 @@ -0,0 +1,277 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Definition of x86 operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from __future__ import absolute_import
+
+import logging
+
+import topi
+from .generic import *
+from .. import op as _op
+from ....schedule import SpecializedCondition
+
+logger = logging.getLogger('strategy')
+
+@schedule_injective.register("cpu")
+def schedule_injective_cpu(attrs, outs, target):
+    """schedule injective ops for x86"""
+    with target:
+        return topi.x86.schedule_injective(outs)
+
+@schedule_reduce.register("cpu")
+def schedule_reduce_cpu(attrs, outs, target):
+    """schedule reduction ops for x86"""
+    with target:
+        return topi.x86.schedule_reduce(outs)
+
+@schedule_concatenate.register("cpu")
+def schedule_concatenate_cpu(attrs, outs, target):
+    """schedule concatenate op for x86"""
+    with target:
+        return topi.x86.schedule_concatenate(outs)
+
+@schedule_pool.register("cpu")
+def schedule_pool_cpu(attrs, outs, target):
+    """schedule pooling ops for x86"""
+    with target:
+        return topi.x86.schedule_pool(outs, attrs.layout)
+
+@schedule_adaptive_pool.register("cpu")
+def schedule_adaptive_pool_cpu(attrs, outs, target):
+    """schedule adaptive pooling ops for x86"""
+    with target:
+        return topi.x86.schedule_adaptive_pool(outs)
+
+@schedule_softmax.register("cpu")
+def schedule_softmax_cpu(attrs, outs, target):
+    """schedule softmax for x86"""
+    with target:
+        return topi.x86.schedule_softmax(outs)
+
+@conv2d_strategy.register("cpu")
+def conv2d_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d x86 strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation_h, dilation_w = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            if topi.x86.is_int8_hw_support(data.dtype, kernel.dtype):
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.conv2d_nchw_int8),
+                    wrap_topi_schedule(topi.x86.schedule_conv2d_nchw_int8))
+            else:
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.conv2d_nchw),
+                    wrap_topi_schedule(topi.x86.schedule_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWIO"
+            logger.warning("For x86 target, NCHW layout is recommended for conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nhwc),
+                wrap_topi_schedule(topi.x86.schedule_conv2d_nhwc))
+        elif layout == "HWCN":
+            assert kernel_layout == "HWIO"
+            logger.warning("For x86 target, NCHW layout is recommended for conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_hwcn),
+                wrap_topi_schedule(topi.generic.schedule_conv2d_hwcn))
+        else:
+            raise RuntimeError("Unsupported conv2d layout {} for cpu".format(layout))
+    elif is_depthwise_conv2d(data.shape, layout, kernel.shape, kernel_layout, groups):
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            channel_multiplier = get_const_tuple(inputs[1].shape)[1]
+            if channel_multiplier == 1:
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.depthwise_conv2d_nchw),
+                    wrap_topi_schedule(topi.x86.schedule_depthwise_conv2d_nchw))
+            else:
+                logger.warning("For x86 target, depthwise_conv2d with channel "
+                               "multiplier greater than 1 is not optimized")
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.nn.depthwise_conv2d_nchw),
+                    wrap_topi_schedule(topi.generic.schedule_depthwise_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWOI"
+            logger.warning("For x86 target, NCHW layout is recommended for depthwise_conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nhwc),
+                wrap_topi_schedule(topi.generic.schedule_depthwise_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported depthwise_conv2d layout {}".format(layout))
+    else: # group_conv2d
+        if layout == 'NCHW':
+            assert kernel_layout == "OIHW"
+            logger.warning("group_conv2d is not optimized for cpu.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.group_conv2d_nchw, has_groups=True),
+                wrap_topi_schedule(topi.generic.schedule_group_conv2d_nchw))
+        else:
+            raise RuntimeError("Unsupported group_conv2d layout {}".format(layout))
+    return strategy
+
+@conv2d_NCHWc_strategy.register("cpu")
+def conv2d_NCHWc_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d_NCHWc x86 strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    if topi.x86.is_int8_hw_support(data.dtype, kernel.dtype):
+        strategy.add_implement(
+            wrap_compute_conv2d(topi.x86.conv2d_NCHWc_int8, True, True),
+            wrap_topi_schedule(topi.x86.schedule_conv2d_NCHWc_int8))
+    else:
+        strategy.add_implement(
+            wrap_compute_conv2d(topi.x86.conv2d_NCHWc, True, True),
+            wrap_topi_schedule(topi.x86.schedule_conv2d_NCHWc))
+    return strategy
+
+@depthwise_conv2d_NCHWc_strategy.register("cpu")
+def depthwise_conv2d_NCHWc_strategy_cpu(attrs, inputs, out_type, target):
+    """depthwise_conv2d x86 strategy"""
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_compute_conv2d(topi.x86.depthwise_conv2d_NCHWc, True, True),
+        wrap_topi_schedule(topi.x86.schedule_depthwise_conv2d_NCHWc))
+    return strategy
+
+@conv2d_transpose_strategy.register("cpu")
+def conv2d_transpose_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d_transpose x86 strategy"""
+    layout = attrs.data_layout
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    assert layout == "NCHW", "only support nchw for now"
+    assert dilation == (1, 1), "not support dilate now"
+    assert groups == 1, "only support groups == 1 for now"
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_comptue_conv2d_transpose(topi.x86.conv2d_transpose_nchw),
+        wrap_topi_schedule(topi.x86.schedule_conv2d_transpose_nchw))
+    return strategy
+
+@conv3d_strategy.register("cpu")
+def conv3d_strategy_cpu(attrs, inputs, out_type, target):
+    """conv3d generic strategy"""
+    strategy = _op.OpStrategy()
+    layout = attrs.data_layout
+    if layout == "NCDHW":
+        logger.warning("conv3d with layout NCDHW is not optimized for cpu.")
 
 Review comment:
   Do you think it makes sense to add these log messages to the generic schedule function itself? The benefit would be not relying on them being manually added and removed when we add new backends or schedules with the downside of not being able to specify which device it's not optimized for. Given that the user is probably aware of what device they're running on, just saying "conv3d with layout NCHW is not optimized on this platform." might be plenty.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r377403887
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Name of the implementation */
+  std::string name;
+  /*! \brief Priority level */
+  int plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
+ public:
+  /*!
+   * \brief Invoke the operator compute function.
+   * \param attrs The attribute of the primitive
+   * \param inputs The input tensors.
+   * \param out_type The output type information.
+   * \return The output compute description of the operator.
+   */
+  Array<te::Tensor> Compute(const Attrs& attrs,
+                            const Array<te::Tensor>& inputs,
+                            const Type& out_type);
+  /*!
+   * \brief Build the computation schedule.
+   * \param attrs The attribute of the node.
+   * \param outs The output tensors.
+   * \param target The build target.
+   * \return The computation schedule.
+   */
+  te::Schedule Schedule(const Attrs& attrs,
+                        const Array<te::Tensor>& outs,
+                        const Target& target);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(OpImplement, ObjectRef, OpImplementNode);
+};
+
+/*!
+ * \brief Specialized implementations for operators under certain conditions.
+ */
+class OpSpecializationNode : public Object {
+ public:
+  /*! \brief List of implementations. */
+  Array<OpImplement> implements;
+  /*! \brief Condition to enable the specialization.
+   *    Could be undefined to represent generic case. */
+  te::SpecializedCondition condition;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("condition", &condition);
+    v->Visit("implements", &implements);
+  }
+
+  static constexpr const char* _type_key = "relay.OpSpecialization";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpSpecializationNode, ExprNode);
+};
+
+/*!
+ * \brief Operator specialization class.
+ */
+class OpSpecialization : public ObjectRef {
+ public:
+  /*!
+   * \brief Add an implementation.
+   * \param compute Compute function
+   * \param schedule Schedule function
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r381088305
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implementations(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.target.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380972090
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    analyzer = tvm.arith.Analyzer()
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = analyzer.canonical_simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
 
 Review comment:
   alternatively, we can use impl

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875263
 
 

 ##########
 File path: python/tvm/relay/op/op.py
 ##########
 @@ -143,39 +144,208 @@ class OpPattern(object):
     OPAQUE = 8
 
 
-def register_schedule(op_name, schedule=None, level=10):
-    """Register schedule function for an op
+@tvm._ffi.register_object("relay.OpImplement")
+class OpImplement(Object):
+    """Operator implementation"""
+    def compute(self, attrs, inputs, out_type):
+        """Call compute function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        inputs : list[tvm.Tensor]
+            The input tensors.
+
+        out_type : relay.Type
+            The output type.
+
+        Returns
+        -------
+        outs : list[tvm.Tensor]
+            The output tensors.
+        """
+        return _OpImplementCompute(self, attrs, inputs, out_type)
+
+    def schedule(self, attrs, outs, target):
+        """Call schedule function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        outs : list[tvm.Tensor]
+            The output tensors.
+
+        target : tvm.Target
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r382400455
 
 

 ##########
 File path: python/tvm/autotvm/record.py
 ##########
 @@ -119,20 +124,29 @@ def decode(row, protocol='json'):
 
     Parameters
     ----------
-    row: str
+    row : str
         a row in the logger file
-    protocol: str
+
+    protocol : str
         log protocol, json or pickle
 
     Returns
     -------
-    input: autotvm.tuner.MeasureInput
-    result: autotvm.tuner.MeasureResult
+    ret : tuple(autotvm.tuner.MeasureInput, autotvm.tuner.MeasureResult), or None
+        The tuple of input and result, or None if input uses old version log format.
     """
     # pylint: disable=unused-variable
+    global _old_version_warning
+
     if protocol == 'json':
         row = json.loads(row)
-        tgt, task_name, task_args, task_kwargs, workload, config = row['i']
+        if 'v' in row and row['v'] == 0.1:
+            if _old_version_warning:
+                logger.warning("AutoTVM log version 0.1 is no longer supported.")
 
 Review comment:
   I think it might be useful in case people haven't updated the tophub, and wonder why the performance is bad. But yes, after we upgraded the tophub, this log message is useless.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] ZihengJiang commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376570164
 
 

 ##########
 File path: python/tvm/autotvm/task/space.py
 ##########
 @@ -944,11 +939,10 @@ def from_json_dict(json_dict):
                 raise RuntimeError("Invalid config knob type: " + knob_type)
             entity_map[str(key)] = entity
 
-        return ConfigEntity(index, code_hash, template_key, entity_map, constraints)
+        return ConfigEntity(index, code_hash, entity_map, constraints)
 
     def __repr__(self):
-        return "%s,%s,%s,%d" % (str(self._entity_map)[12:-1], self.template_key,
-                                self.code_hash, self.index)
+        return "%s,%s,%d" % (str(self._entity_map)[12:-1], self.code_hash, self.index)
 
 Review comment:
   Will this change our autotvm log format?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Name of the implementation */
+  std::string name;
+  /*! \brief Priority level */
+  int plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
 
 Review comment:
   Move related classes to relay/op_strategy.h

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] junrushao1994 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
junrushao1994 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r375566300
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,14 +216,182 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
                      const Array<te::Tensor>& inputs,
                      const Array<IndexExpr>& out_ndims)>;
 
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Priority level */
+  Integer plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
+ public:
+  /*! \brief default constructor  */
+  OpImplement() {}
+  /*! \brief constructor from node pointer */
+  explicit OpImplement(ObjectPtr<Object> n) : ObjectRef(n) {}
+  /*!
+   * \brief access the internal node container
+   * \return the pointer to the internal node container
+   */
+  inline const OpImplementNode* operator->() const;
 
 Review comment:
   Shall we use `TVM_DEFINE_OBJECT_REF_METHODS` 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/relay/op/op.py
 ##########
 @@ -143,39 +144,208 @@ class OpPattern(object):
     OPAQUE = 8
 
 
-def register_schedule(op_name, schedule=None, level=10):
-    """Register schedule function for an op
+@tvm._ffi.register_object("relay.OpImplement")
+class OpImplement(Object):
+    """Operator implementation"""
+    def compute(self, attrs, inputs, out_type):
+        """Call compute function.
+
+        Parameters
+        ----------
+        attrs : Attrs
+            Op attributes.
+
+        inputs : list[tvm.Tensor]
 
 Review comment:
   tvm.tensor.Tensor

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379875286
 
 

 ##########
 File path: src/relay/op/tensor/transform.cc
 ##########
 @@ -1688,9 +1674,8 @@ bool SqueezeRel(const Array<Type>& types,
 }
 
 Array<te::Tensor> SqueezeCompute(const Attrs& attrs,
-                             const Array<te::Tensor>& inputs,
-                             const Type& out_type,
-                             const Target& target) {
+                                  const Array<te::Tensor>& inputs,
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r377309086
 
 

 ##########
 File path: include/tvm/te/schedule.h
 ##########
 @@ -742,6 +743,55 @@ class SingletonNode : public IterVarRelationNode {
   TVM_DECLARE_FINAL_OBJECT_INFO(SingletonNode, IterVarRelationNode);
 };
 
+class SpecializedConditionNode;
+
+/*!
+ * \brief Specialized condition to enable op specialization
+ */
+class SpecializedCondition : public ObjectRef {
+ public:
+  SpecializedCondition() {}
+  explicit SpecializedCondition(ObjectPtr<Object> n) : ObjectRef(n) {}
+  /*!
+   * \brief Get the current specialized condition.
+   * \return The current specialized condition.
+   */
+  TVM_DLL static SpecializedCondition Current();
+
+  const SpecializedConditionNode* operator->() const;
+
+  using ContainerType = SpecializedConditionNode;
+  class Internal;
+ private:
+  // enable with syntax.
+  friend class Internal;
+  friend class With<SpecializedCondition>;
+  /*! \brief Push a new specialized condition onto the thread local stack. */
+  TVM_DLL void EnterWithScope();
+  /*! \brief Pop a specialized condition off the thread local context stack. */
+  TVM_DLL void ExitWithScope();
+};
+
+/*! \brief Container for specialization conditions. */
+class SpecializedConditionNode : public Object {
+ public:
+  /*!
+   * \brief List of conditions in conjunctive joint form (CNF).
+   *   Each condition should be a simple expression, e.g., n > 16, m % 8 == 0, etc.,
+   *   where n, m are tvm::Var that represents a dimension in the tensor shape.
+   */
+  Array<PrimExpr> clauses;
+
+  void VisitAttrs(AttrVisitor* v) {
+    v->Visit("clauses", &clauses);
+  }
+
+  static SpecializedCondition make(Array<PrimExpr> conditions);
+
+  static constexpr const char* _type_key = "SpecializedCondition";
 
 Review comment:
   It seems all type keys in te didn't add "te" in the type 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


With regards,
Apache Git Services

[GitHub] [tvm] apivovarov commented on pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
apivovarov commented on pull request #4644:
URL: https://github.com/apache/tvm/pull/4644#issuecomment-963638579


   There are couple TODOs related to PR4644 - `TODO enforce 4-way padding in topi/nn/conv2d after #4644 merged`
   - https://github.com/apache/tvm/blob/d11bdcd3ad0717b8e38ba769e849d6a6afe6415e/python/tvm/relay/op/nn/nn.py#L220
   - https://github.com/apache/tvm/blob/4ffbdcd0aaed4f382f06c6a9e2b2d048b6abdaa9/python/tvm/relay/qnn/op/qnn.py#L348


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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



[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r377403861
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +85,317 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implements:
+                    ret.append(impl)
+        else:
+            for impl in spec.implements:
+                ret.append(impl)
+    return ret
+
+
+def select_implement(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implement from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op that has symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
+        The best op implementation and the corresponding output tensors.
+    """
+    all_impls = get_valid_implements(op, attrs, inputs, out_type, target)
+
+    best_plevel_impl = None
+    for impl in all_impls:
+        if best_plevel_impl is None or impl.plevel > best_plevel_impl.plevel:
+            best_plevel_impl = impl
+    if not use_autotvm:
+        outs = best_plevel_impl.compute(attrs, inputs, out_type)
+        return best_plevel_impl, outs
+
+    outputs = {}
+    best_autotvm_impl = None
+    best_cfg = None
+    dispatch_ctx = autotvm.task.DispatchContext.current
+    for impl in all_impls:
+        outs = impl.compute(attrs, inputs, out_type)
+        outputs[impl] = outs
+        workload = autotvm.task.get_workload(outs)
+        if workload is None:
+            continue
+        cfg = dispatch_ctx.query(target, workload)
+        if cfg.cost is None:
+            # It's a fallback config
+            continue
+        if best_cfg is None or best_cfg.cost > cfg.cost:
+            best_autotvm_impl = impl
+            best_cfg = cfg
+    if best_autotvm_impl:
+        return best_autotvm_impl, outputs[best_autotvm_impl]
+    return best_plevel_impl, outputs[best_plevel_impl]
+
+
+class ScheduleGetter(ExprVisitor):
 
 Review comment:
   Now only port part of ScheduleGetter to 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 edited a comment on issue #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 edited a comment on issue #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-579944529
 
 
   I've added the strategy for all ops. We can start to review this PR since it's huge. Could you help review it?
   @tqchen @kevinthesun @comaniac @masahi @MarisaKirisame @hlu1 @yzhliu @zhiics @ZihengJiang @merrymercy @vinx13 @FrozenGene @jroesch @jwfromm 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373142456
 
 

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +149,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE2 = {}
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r381065750
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implementations(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.target.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    analyzer = tvm.arith.Analyzer()
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = analyzer.canonical_simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implementations:
+                    ret.append(impl)
+        else:
+            for impl in spec.implementations:
+                ret.append(impl)
+    return ret
+
+
+def select_implementation(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implementation from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.target.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
 
 Review comment:
   OpImplementaion

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,190 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implements(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = tvm.ir_pass.Simplify(clause)
 
 Review comment:
   Please use tvm.arith's Analyzer instead for simplifications, as they are more direct approaches, see https://github.com/apache/incubator-tvm/blob/master/python/tvm/arith.py#L164

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376142732
 
 

 ##########
 File path: python/tvm/relay/op/strategy/x86.py
 ##########
 @@ -0,0 +1,277 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Definition of x86 operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from __future__ import absolute_import
+
+import logging
+
+import topi
+from .generic import *
+from .. import op as _op
+from ....schedule import SpecializedCondition
+
+logger = logging.getLogger('strategy')
+
+@schedule_injective.register("cpu")
+def schedule_injective_cpu(attrs, outs, target):
+    """schedule injective ops for x86"""
+    with target:
+        return topi.x86.schedule_injective(outs)
+
+@schedule_reduce.register("cpu")
+def schedule_reduce_cpu(attrs, outs, target):
+    """schedule reduction ops for x86"""
+    with target:
+        return topi.x86.schedule_reduce(outs)
+
+@schedule_concatenate.register("cpu")
+def schedule_concatenate_cpu(attrs, outs, target):
+    """schedule concatenate op for x86"""
+    with target:
+        return topi.x86.schedule_concatenate(outs)
+
+@schedule_pool.register("cpu")
+def schedule_pool_cpu(attrs, outs, target):
+    """schedule pooling ops for x86"""
+    with target:
+        return topi.x86.schedule_pool(outs, attrs.layout)
+
+@schedule_adaptive_pool.register("cpu")
+def schedule_adaptive_pool_cpu(attrs, outs, target):
+    """schedule adaptive pooling ops for x86"""
+    with target:
+        return topi.x86.schedule_adaptive_pool(outs)
+
+@schedule_softmax.register("cpu")
+def schedule_softmax_cpu(attrs, outs, target):
+    """schedule softmax for x86"""
+    with target:
+        return topi.x86.schedule_softmax(outs)
+
+@conv2d_strategy.register("cpu")
+def conv2d_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d x86 strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation_h, dilation_w = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            if topi.x86.is_int8_hw_support(data.dtype, kernel.dtype):
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.conv2d_nchw_int8),
+                    wrap_topi_schedule(topi.x86.schedule_conv2d_nchw_int8))
+            else:
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.conv2d_nchw),
+                    wrap_topi_schedule(topi.x86.schedule_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWIO"
+            logger.warning("For x86 target, NCHW layout is recommended for conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nhwc),
+                wrap_topi_schedule(topi.x86.schedule_conv2d_nhwc))
+        elif layout == "HWCN":
+            assert kernel_layout == "HWIO"
+            logger.warning("For x86 target, NCHW layout is recommended for conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_hwcn),
+                wrap_topi_schedule(topi.generic.schedule_conv2d_hwcn))
+        else:
+            raise RuntimeError("Unsupported conv2d layout {} for cpu".format(layout))
+    elif is_depthwise_conv2d(data.shape, layout, kernel.shape, kernel_layout, groups):
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            channel_multiplier = get_const_tuple(inputs[1].shape)[1]
+            if channel_multiplier == 1:
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.depthwise_conv2d_nchw),
+                    wrap_topi_schedule(topi.x86.schedule_depthwise_conv2d_nchw))
+            else:
+                logger.warning("For x86 target, depthwise_conv2d with channel "
+                               "multiplier greater than 1 is not optimized")
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.nn.depthwise_conv2d_nchw),
+                    wrap_topi_schedule(topi.generic.schedule_depthwise_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWOI"
+            logger.warning("For x86 target, NCHW layout is recommended for depthwise_conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nhwc),
+                wrap_topi_schedule(topi.generic.schedule_depthwise_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported depthwise_conv2d layout {}".format(layout))
+    else: # group_conv2d
+        if layout == 'NCHW':
+            assert kernel_layout == "OIHW"
+            logger.warning("group_conv2d is not optimized for cpu.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.group_conv2d_nchw, has_groups=True),
+                wrap_topi_schedule(topi.generic.schedule_group_conv2d_nchw))
+        else:
+            raise RuntimeError("Unsupported group_conv2d layout {}".format(layout))
+    return strategy
+
+@conv2d_NCHWc_strategy.register("cpu")
+def conv2d_NCHWc_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d_NCHWc x86 strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    if topi.x86.is_int8_hw_support(data.dtype, kernel.dtype):
+        strategy.add_implement(
+            wrap_compute_conv2d(topi.x86.conv2d_NCHWc_int8, True, True),
+            wrap_topi_schedule(topi.x86.schedule_conv2d_NCHWc_int8))
+    else:
+        strategy.add_implement(
+            wrap_compute_conv2d(topi.x86.conv2d_NCHWc, True, True),
+            wrap_topi_schedule(topi.x86.schedule_conv2d_NCHWc))
+    return strategy
+
+@depthwise_conv2d_NCHWc_strategy.register("cpu")
+def depthwise_conv2d_NCHWc_strategy_cpu(attrs, inputs, out_type, target):
+    """depthwise_conv2d x86 strategy"""
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_compute_conv2d(topi.x86.depthwise_conv2d_NCHWc, True, True),
+        wrap_topi_schedule(topi.x86.schedule_depthwise_conv2d_NCHWc))
+    return strategy
+
+@conv2d_transpose_strategy.register("cpu")
+def conv2d_transpose_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d_transpose x86 strategy"""
+    layout = attrs.data_layout
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    assert layout == "NCHW", "only support nchw for now"
+    assert dilation == (1, 1), "not support dilate now"
+    assert groups == 1, "only support groups == 1 for now"
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_comptue_conv2d_transpose(topi.x86.conv2d_transpose_nchw),
+        wrap_topi_schedule(topi.x86.schedule_conv2d_transpose_nchw))
+    return strategy
+
+@conv3d_strategy.register("cpu")
+def conv3d_strategy_cpu(attrs, inputs, out_type, target):
+    """conv3d generic strategy"""
+    strategy = _op.OpStrategy()
+    layout = attrs.data_layout
+    if layout == "NCDHW":
+        logger.warning("conv3d with layout NCDHW is not optimized for cpu.")
 
 Review comment:
   added warning message for generic strategies except for these simple ops that might not be perf critical.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on issue #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on issue #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-587915240
 
 
   @tqchen Could you review this again?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379980105
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
 
 Review comment:
   Just OpImplNode?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] yzhliu commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
yzhliu commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380275069
 
 

 ##########
 File path: topi/python/topi/cuda/conv3d.py
 ##########
 @@ -126,24 +78,55 @@ def schedule_conv3d_ncdhw_cuda(cfg, outs):
     s: Schedule
         The computation schedule for conv2d.
     """
-    target = tvm.target.Target.current()
-    if 'cudnn' in target.libs:
-        return generic.schedule_extern(outs)
-
     outs = [outs] if isinstance(outs, tvm.tensor.Tensor) else outs
     s = tvm.create_schedule([x.op for x in outs])
 
     def _callback(op):
         if op.tag == 'conv3d_ncdhw':
-            schedule_direct_3d_cuda(cfg, s, op.output(0))
+            schedule_direct_conv3d_cuda(cfg, s, op.output(0), "NCDHW",
+                                        "conv3d_ncdhw.cuda")
 
     traverse_inline(s, outs[0].op, _callback)
     return s
 
 
-@autotvm.register_topi_schedule(generic.schedule_conv3d_ndhwc, ["cuda", "gpu"],
-                                ["direct"])
-def schedule_conv3d_ndhwc_cuda(cfg, outs):
+@autotvm.register_topi_compute("conv3d_ndhwc.cuda")
+def conv3d_ndhwc(cfg, data, kernel, strides, padding, dilation, out_dtype='float32'):
+    """Conv3D operator for cuda backend.
+
+    Parameters
+    ----------
+    cfg: ConfigEntity
+        The config for this template
+
+    data : tvm.Tensor
+        5-D with shape [batch, in_channel, in_depth, in_height, in_width]
 
 Review comment:
   should be ndhwc ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380302988
 
 

 ##########
 File path: topi/python/topi/cuda/conv3d.py
 ##########
 @@ -126,24 +78,55 @@ def schedule_conv3d_ncdhw_cuda(cfg, outs):
     s: Schedule
         The computation schedule for conv2d.
     """
-    target = tvm.target.Target.current()
-    if 'cudnn' in target.libs:
-        return generic.schedule_extern(outs)
-
     outs = [outs] if isinstance(outs, tvm.tensor.Tensor) else outs
     s = tvm.create_schedule([x.op for x in outs])
 
     def _callback(op):
         if op.tag == 'conv3d_ncdhw':
-            schedule_direct_3d_cuda(cfg, s, op.output(0))
+            schedule_direct_conv3d_cuda(cfg, s, op.output(0), "NCDHW",
+                                        "conv3d_ncdhw.cuda")
 
     traverse_inline(s, outs[0].op, _callback)
     return s
 
 
-@autotvm.register_topi_schedule(generic.schedule_conv3d_ndhwc, ["cuda", "gpu"],
-                                ["direct"])
-def schedule_conv3d_ndhwc_cuda(cfg, outs):
+@autotvm.register_topi_compute("conv3d_ndhwc.cuda")
+def conv3d_ndhwc(cfg, data, kernel, strides, padding, dilation, out_dtype='float32'):
+    """Conv3D operator for cuda backend.
+
+    Parameters
+    ----------
+    cfg: ConfigEntity
+        The config for this template
+
+    data : tvm.Tensor
+        5-D with shape [batch, in_channel, in_depth, in_height, in_width]
 
 Review comment:
   fixed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380971807
 
 

 ##########
 File path: docs/api/python/dev.rst
 ##########
 @@ -28,9 +28,9 @@ tvm.expr
    :members:
    :undoc-members:
 
-tvm.codegen
-~~~~~~~~~~~
-.. automodule:: tvm.codegen
+tvm.target.codegen
+~~~~~~~~~~~~~~~~~~
 
 Review comment:
   can ignore changes for now, superceded by https://github.com/apache/incubator-tvm/pull/4907

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: src/relay/op/tensor/transform.cc
 ##########
 @@ -1688,9 +1674,8 @@ bool SqueezeRel(const Array<Type>& types,
 }
 
 Array<te::Tensor> SqueezeCompute(const Attrs& attrs,
-                             const Array<te::Tensor>& inputs,
-                             const Type& out_type,
-                             const Target& target) {
+                                  const Array<te::Tensor>& inputs,
 
 Review comment:
   align

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373142390
 
 

 ##########
 File path: python/tvm/relay/quantize/_annotate.py
 ##########
 @@ -53,11 +53,11 @@ def simulated_quantize_compute(attrs, inputs, out_type, target):
     return [rdata]
 
 
-_reg.register_schedule("relay.op.annotation.simulated_quantize",
-                       _reg.schedule_injective)
+# _reg.register_schedule("relay.op.annotation.simulated_quantize",
 
 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: topi/python/topi/intel_graphics/conv2d.py
 ##########
 @@ -159,108 +147,59 @@ def tile_and_bind3d(s, tensor, z, y, x, z_factor=2, y_factor=None, x_factor=None
 # We define schedule template in this function instead of
 # declaration function since actual input arguments need
 # to be altered by the schedule selected.
-@autotvm.task.register("topi_intel_graphics_conv2d_NCHWc")
-def __topi_nn_conv2d_NCHWc(*args, **kwargs):
-    assert not kwargs, "Do not support kwargs in template function call"
-    data, kernel, strides, padding, dilation, layout, dtype = deserialize_args(args)
-    raw_data_shape = get_const_tuple(data.shape)
-    raw_kernel_shape = get_const_tuple(kernel.shape)
-
-    # get config here
-    cfg = get_config()
-    _create_schedule_template(cfg, data, kernel, strides, padding, dilation, layout)
-    cfg.add_flop(1)
-
-    # change shape with the value in config
-    ic_bn = cfg["tile_ic"].val if hasattr(cfg["tile_ic"], "val") else cfg["tile_ic"].size[-1]
-    oc_bn = cfg["tile_oc"].val if hasattr(cfg["tile_oc"], "val") else cfg["tile_oc"].size[-1]
-
-    new_data_shape = (raw_data_shape[0], raw_data_shape[1] // ic_bn,
-                      raw_data_shape[2], raw_data_shape[3], ic_bn)
-    new_kernel_shape = (raw_kernel_shape[0] // oc_bn, raw_kernel_shape[1] // ic_bn,
-                        raw_kernel_shape[2], raw_kernel_shape[3], ic_bn, oc_bn)
-    new_data = tvm.placeholder(new_data_shape, data.dtype)
-    new_kernel = tvm.placeholder(new_kernel_shape, kernel.dtype)
-
-    C = _decl_cl_spatialpack_NCHWc(cfg, new_data, new_kernel, strides, padding, dilation, dtype)
-    s = _schedule_conv2d_NCHWc(cfg, [C])
-
-    return s, [new_data, new_kernel, C]
-
-@conv2d_alter_layout.register(["intel_graphics"])
-def _alter_conv2d_layout(attrs, inputs, tinfo, F):
-    copy_inputs = list(inputs)
-    new_attrs = {k : attrs[k] for k in attrs.keys()}
-
-    if F.__name__ == 'tvm.relay.op':
-        # Derive channels for frontends (e.g ONNX) that miss "channel" field.
-        new_attrs["channels"] = inputs[1].checked_type.shape[attrs['kernel_layout'].index('O')]
-
-    data, kernel = tinfo[0], tinfo[1]
-    batch_size, in_channel, height, width = get_const_tuple(data.shape)
+# @autotvm.task.register("topi_intel_graphics_conv2d_NCHWc")
 
 Review comment:
   Remove or add a 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380383331
 
 

 ##########
 File path: python/tvm/autotvm/task/relay_integration.py
 ##########
 @@ -67,27 +65,22 @@ def extract_from_program(mod, params, ops, target, target_host=None,
         The module or function to tune
     params: dict of str to numpy array
         The associated parameters of the program
-    ops: List of relay op
-        List of relay ops to be tuned
     target: tvm.target.Target
         The compilation target
     target_host: tvm.target.Target
         The host compilation target
-    template_keys: dict of topi op to str
-        The tuning template keys map for schedules, default to None.
-        Example: {topi.nn.conv2d: 'direct'}
+    ops: List of relay.op.Op
+        List of relay ops to be tuned
 
     Returns
     -------
     task: Array of autotvm.task.Task
         collected tasks
     """
-    return extract_from_multiple_program([mod], [params], ops, target, target_host,
-                                         template_keys)
+    return extract_from_multiple_program([mod], [params], target, target_host, ops)
 
 
-def extract_from_multiple_program(mods, params, ops, target, target_host=None,
-                                  template_keys=None):
+def extract_from_multiple_program(mods, params, target, target_host=None, ops=None):
 
 Review comment:
   Yes, that's the current behavior. I'll update the doc to show 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380382775
 
 

 ##########
 File path: python/tvm/autotvm/task/relay_integration.py
 ##########
 @@ -67,27 +65,22 @@ def extract_from_program(mod, params, ops, target, target_host=None,
         The module or function to tune
     params: dict of str to numpy array
         The associated parameters of the program
-    ops: List of relay op
-        List of relay ops to be tuned
     target: tvm.target.Target
         The compilation target
     target_host: tvm.target.Target
         The host compilation target
-    template_keys: dict of topi op to str
-        The tuning template keys map for schedules, default to None.
-        Example: {topi.nn.conv2d: 'direct'}
+    ops: List of relay.op.Op
+        List of relay ops to be tuned
 
     Returns
     -------
     task: Array of autotvm.task.Task
         collected tasks
     """
-    return extract_from_multiple_program([mod], [params], ops, target, target_host,
-                                         template_keys)
+    return extract_from_multiple_program([mod], [params], target, target_host, ops)
 
 
-def extract_from_multiple_program(mods, params, ops, target, target_host=None,
-                                  template_keys=None):
+def extract_from_multiple_program(mods, params, target, target_host=None, ops=None):
 
 Review comment:
   Would that be better if we include all ops by default if `ops=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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r373139677
 
 

 ##########
 File path: python/tvm/autotvm/task/dispatcher.py
 ##########
 @@ -481,8 +412,12 @@ def _query_inside(self, target, workload):
         """
         if self._counter < len(self._records):
             cfg = self._records[self._counter][0].config
+            wkl = self._records[self._counter][0].task.workload
+            if workload is not None:
+                assert wkl == workload
             self._counter += 1
-            self.update(target, workload, cfg)
+            self.update(target, wkl, cfg)
+            cfg.workload = wkl
 
 Review comment:
   This is only specific to `ApplyGraphBest`. The reason is complicated. Because `ApplyGraphBest` relies on the order of query, we cannot use `relay.backend.compile_engine.select_implement` to collect the autotvm workload as it may query more than once. Therefore, this is a temporary work around that we sneak in the workload in the return cfg. We can remove this part of logic after we make `ApplyGraphBest` no longer relies on the query order.
   @kevinthesun 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380296359
 
 

 ##########
 File path: topi/python/topi/arm_cpu/__init__.py
 ##########
 @@ -14,13 +14,14 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
+# pylint: disable=wildcard-import
 """Schedule for ARM CPU"""
 
-from . import conv2d
-from . import depthwise_conv2d
-from . import conv2d_transpose
-from . import conv2d_int8
-from . import bitserial_conv2d
-from . import bitserial_dense
-from . import injective
+from .conv2d import *
+from .depthwise_conv2d import *
+from .conv2d_transpose import *
+from .conv2d_int8 import *
+from . import conv2d_alter_op
+from .bitserial_conv2d import *
+from .bitserial_dense import *
+from .injective import *
 
 Review comment:
   Just easier to get the compute and schedule function, and consistent with other 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] ZihengJiang commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376651491
 
 

 ##########
 File path: python/tvm/autotvm/task/space.py
 ##########
 @@ -944,11 +939,10 @@ def from_json_dict(json_dict):
                 raise RuntimeError("Invalid config knob type: " + knob_type)
             entity_map[str(key)] = entity
 
-        return ConfigEntity(index, code_hash, template_key, entity_map, constraints)
+        return ConfigEntity(index, code_hash, entity_map, constraints)
 
     def __repr__(self):
-        return "%s,%s,%s,%d" % (str(self._entity_map)[12:-1], self.template_key,
-                                self.code_hash, self.index)
+        return "%s,%s,%d" % (str(self._entity_map)[12:-1], self.code_hash, self.index)
 
 Review comment:
   Then we may need to bump the log format version, if we cannot ensure the compatibility, we should raise error when read the old version. Also, we can take this opportunity to fix our previous design, like changing the shortcut `i`, `r`, `v` to the full word: https://github.com/apache/incubator-tvm/blob/master/python/tvm/autotvm/record.py#L91

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/autotvm/task/topi_integration.py
 ##########
 @@ -76,250 +40,49 @@ class TaskExtractEnv:
     registered = None
 
     def __init__(self, allow_duplicate=False):
-        # pylint: disable=import-outside-toplevel
-        import topi
-
-        # topi compute -> autotvm task name
-        self.topi_to_task = {
-            topi.nn.conv2d: "topi_nn_conv2d",
-            topi.nn.depthwise_conv2d_nchw: "topi_nn_depthwise_conv2d_nchw",
-            topi.nn.group_conv2d_nchw: "topi_nn_group_conv2d_nchw",
-            topi.nn.conv2d_transpose_nchw: "topi_nn_conv2d_transpose_nchw",
-            topi.nn.conv2d_NCHWc: "topi_x86_conv2d_NCHWc",
-            topi.nn.conv2d_NCHWc_int8: "topi_x86_conv2d_NCHWc_int8",
-            topi.nn.dense: "topi_nn_dense",
-            topi.nn.batch_matmul: "topi_nn_batch_matmul",
-            topi.nn.bitserial_conv2d_nchw: "topi_nn_bitserial_conv2d_nchw",
-            topi.nn.bitserial_conv2d_nhwc: "topi_nn_bitserial_conv2d_nhwc",
-            topi.nn.bitserial_dense: "topi_nn_bitserial_dense",
-            topi.nn.deformable_conv2d_nchw: "topi_nn_deformable_conv2d_nchw",
-            topi.nn.conv1d_transpose_ncw: "topi_nn_conv1d_transpose_ncw",
-            topi.nn.conv3d: "topi_nn_conv3d",
-        }
-
-        self.topi_to_schedule = {
-            topi.nn.conv2d: [topi.generic.schedule_conv2d_nchw,
-                             topi.generic.schedule_conv2d_nhwc],
-            topi.nn.depthwise_conv2d_nchw: [topi.generic.schedule_depthwise_conv2d_nchw,
-                                            topi.generic.schedule_depthwise_conv2d_nhwc],
-            topi.nn.group_conv2d_nchw: [topi.generic.schedule_group_conv2d_nchw],
-            topi.nn.conv2d_transpose_nchw: [topi.generic.schedule_conv2d_transpose_nchw],
-            topi.nn.conv2d_NCHWc: [topi.generic.schedule_conv2d_NCHWc],
-            topi.nn.conv2d_NCHWc_int8: [topi.generic.schedule_conv2d_NCHWc_int8],
-            topi.nn.dense: [topi.generic.schedule_dense],
-            topi.nn.batch_matmul: [topi.generic.schedule_batch_matmul],
-            topi.nn.bitserial_conv2d_nchw: [topi.generic.schedule_bitserial_conv2d_nchw],
-            topi.nn.bitserial_conv2d_nhwc: [topi.generic.schedule_bitserial_conv2d_nhwc],
-            topi.nn.bitserial_dense: [topi.generic.schedule_bitserial_dense],
-            topi.nn.deformable_conv2d_nchw: [topi.generic.schedule_deformable_conv2d_nchw],
-            topi.nn.conv1d_transpose_ncw: [topi.generic.schedule_conv1d_transpose_ncw],
-            topi.nn.conv3d: [topi.generic.schedule_conv3d_ndhwc],
-        }
-
-        # function reflection for tracing
-        self.func_to_reflection = {
-            topi.nn.conv2d:                 lambda x: setattr(topi.nn, 'conv2d', x),
-            topi.nn.conv2d_NCHWc:           lambda x: setattr(topi.nn, 'conv2d_NCHWc', x),
-            topi.nn.conv2d_NCHWc_int8:      lambda x: setattr(topi.nn, 'conv2d_NCHWc_int8', x),
-            topi.nn.depthwise_conv2d_nchw:  lambda x: setattr(topi.nn, 'depthwise_conv2d_nchw', x),
-            topi.nn.group_conv2d_nchw:      lambda x: setattr(topi.nn, 'group_conv2d_nchw', x),
-            topi.nn.conv2d_transpose_nchw:  lambda x: setattr(topi.nn, 'conv2d_transpose_nchw', x),
-            topi.nn.dense:                  lambda x: setattr(topi.nn, 'dense', x),
-            topi.nn.batch_matmul:           lambda x: setattr(topi.nn, 'batch_matmul', x),
-            topi.nn.bitserial_conv2d_nchw:  lambda x: setattr(topi.nn, 'bitserial_conv2d_nchw', x),
-            topi.nn.bitserial_conv2d_nhwc:  lambda x: setattr(topi.nn, 'bitserial_conv2d_nhwc', x),
-            topi.nn.bitserial_dense:        lambda x: setattr(topi.nn, 'bitserial_dense', x),
-            topi.nn.deformable_conv2d_nchw: lambda x: setattr(topi.nn, 'deformable_conv2d_nchw', x),
-            topi.nn.conv1d_transpose_ncw:   lambda x: setattr(topi.nn, 'conv1d_transpose_ncw', x),
-            topi.nn.conv3d:                 lambda x: setattr(topi.nn, 'conv3d', x),
-        }
-
         self.allow_duplicate = allow_duplicate
-        self._register_topi_task()
         self.task_collection = []
-        self.wanted_topi_funcs = list(self.topi_to_task.keys())
+        self.wanted_relay_ops = None
         self.modified_funcs = []
+        self.tracing = False
 
     def __enter__(self):
         self.task_collection = []
-        self.modified_funcs = []
-
-        for topi_compute in self.wanted_topi_funcs:
-            def _local_scope(compute_func):
-                """start a scope to hold the local function in for loop"""
-
-                def _tracing_wrapper(*args, **kwargs):
-                    assert not kwargs, "Do not support extracting tuning tasks when " \
-                                       "kwargs is used in TOPI function call. " \
-                                       "Please modify it to use only positional args."
-                    key = (self.topi_to_task[compute_func], serialize_args(args))
-                    if self.allow_duplicate or key not in self.task_collection:
-                        self.task_collection.append(key)
-
-                    return compute_func(*args, **kwargs)
-
-                self.func_to_reflection[compute_func](_tracing_wrapper)
-                self.modified_funcs.append(compute_func)
-
-            _local_scope(topi_compute)
+        self.tracing = True
 
         return self
 
     def __exit__(self, exc_type, exc_val, exc_tb):
-        # revert modification
-        for func in self.modified_funcs:
-            self.func_to_reflection[func](func)
-
-    def _register_topi_task(self):
-        """register tuning wrapper for topi function"""
-        # pylint: disable=import-outside-toplevel
-        import topi
-
-        # Avoid double registration for certain targets
-        if TaskExtractEnv.registered:
-            return
-        TaskExtractEnv.registered = True
-
-        # Tuning wrapper for topi functions
-        @register("topi_nn_conv2d")
-        def _topi_nn_conv2d(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            layout = args[-2]
-            C = topi.nn.conv2d(*args, **kwargs)
-            if layout == 'NCHW':
-                s = topi.generic.schedule_conv2d_nchw([C])
-            elif layout == 'HWCN':
-                s = topi.generic.schedule_conv2d_hwcn([C])
-            elif layout == 'NHWC':
-                s = topi.generic.schedule_conv2d_nhwc([C])
-            else:
-                raise ValueError("Unsupported layout {}".format(layout))
-            return s, [A, W, C]
-
-        @register("topi_nn_depthwise_conv2d_nchw")
-        def _topi_nn_depthwise_conv2d_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.depthwise_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.schedule_depthwise_conv2d_nchw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_group_conv2d_nchw")
-        def _topi_nn_group_conv2d_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.group_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.schedule_group_conv2d_nchw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_conv2d_transpose_nchw")
-        def _topi_nn_conv2d_transpose_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv2d_transpose_nchw(*args, **kwargs)
-            s = topi.generic.schedule_conv2d_transpose_nchw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_conv1d_transpose_ncw")
-        def _topi_nn_conv1d_transpose_ncw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv1d_transpose_ncw(*args, **kwargs)
-            s = topi.generic.schedule_conv1d_transpose_ncw([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_conv3d")
-        def _topi_nn_conv3d(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv3d(*args, **kwargs)
-            s = topi.generic.schedule_conv3d_ndhwc([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_dense")
-        def _topi_nn_dense(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            if len(args) > 2:
-                data, weight, bias = args[:3]
-            else:
-                data, weight = args
-                bias = None
-            C = topi.nn.dense(*args, **kwargs)
-            s = topi.generic.schedule_dense([C])
-            if bias is not None:
-                return s, [data, weight, bias, C]
-            return s, [data, weight, C]
-
-        @register("topi_nn_batch_matmul")
-        def _topi_nn_batch_matmul(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, B = args
-            C = topi.nn.batch_matmul(A, B)
-            s = topi.generic.schedule_batch_matmul([C])
-            return s, [A, B, C]
-
-        @register("topi_nn_bitserial_conv2d_nhwc")
-        def _topi_bitserial_conv2d_nhwc(*args, **kwargs):
-            args = deserialize_args(args)
-            C = topi.nn.bitserial_conv2d_nhwc(*args, **kwargs)
-            s = topi.generic.nn.schedule_bitserial_conv2d_nhwc([C])
-            A, W = args[:2]
-            return s, [A, W, C]
-
-        @register("topi_nn_bitserial_conv2d_nchw")
-        def _topi_bitserial_conv2d_nchw(*args, **kwargs):
-            args = deserialize_args(args)
-            C = topi.nn.bitserial_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.nn.schedule_bitserial_conv2d_nchw([C])
-            A, W = args[:2]
-            return s, [A, W, C]
-
-        @register("topi_nn_bitserial_dense")
-        def _topi_nn_bitserial_dense(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.bitserial_dense(*args, **kwargs)
-            s = topi.generic.schedule_bitserial_dense([C])
-            return s, [A, W, C]
-
-        @register("topi_nn_deformable_conv2d_nchw")
-        def _topi_nn_deformable_conv2d_nchw(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, Offset, W = args[:3]
-            C = topi.nn.deformable_conv2d_nchw(*args, **kwargs)
-            s = topi.generic.schedule_deformable_conv2d_nchw([C])
-            return s, [A, Offset, W, C]
-
-        @register("topi_nn_conv2d_NCHWc")
-        def _topi_nn_conv2d_NCHWc(*args, **kwargs):
-            assert not kwargs, "Do not support kwargs in template function call"
-            args = deserialize_args(args)
-            A, W = args[:2]
-            C = topi.nn.conv2d_NCHWc(*args, **kwargs)
-            s = topi.generic.schedule_conv2d_NCHWc([C])
-            return s, [A, W, C]
+        self.tracing = False
 
-    def reset(self, wanted_topi_funcs):
+    def reset(self, wanted_relay_ops=None):
         """Reset task collections
 
         Parameters
         ----------
-        wanted_topi_funcs: List of function
-            The topi function to be extracted
+        wanted_relay_ops: List of relay.op.Op
+            The relay ops to be extracted
         """
         self.task_collection = []
-        self.wanted_topi_funcs = wanted_topi_funcs
+        self.wanted_relay_ops = wanted_relay_ops
+
+    def add_task(self, task_name, args):
+        """Add AutoTVM task
+
+        Parameters
+        ----------
+        task_name: str
+            AutoTVM task name.
+
+        args: tuple
+            Arguments to the TOPI function.
+
+        cond: SpecializedCondition
 
 Review comment:
   Looks this 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r375982082
 
 

 ##########
 File path: python/tvm/relay/op/strategy/x86.py
 ##########
 @@ -0,0 +1,277 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Definition of x86 operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from __future__ import absolute_import
+
+import logging
+
+import topi
+from .generic import *
+from .. import op as _op
+from ....schedule import SpecializedCondition
+
+logger = logging.getLogger('strategy')
+
+@schedule_injective.register("cpu")
+def schedule_injective_cpu(attrs, outs, target):
+    """schedule injective ops for x86"""
+    with target:
+        return topi.x86.schedule_injective(outs)
+
+@schedule_reduce.register("cpu")
+def schedule_reduce_cpu(attrs, outs, target):
+    """schedule reduction ops for x86"""
+    with target:
+        return topi.x86.schedule_reduce(outs)
+
+@schedule_concatenate.register("cpu")
+def schedule_concatenate_cpu(attrs, outs, target):
+    """schedule concatenate op for x86"""
+    with target:
+        return topi.x86.schedule_concatenate(outs)
+
+@schedule_pool.register("cpu")
+def schedule_pool_cpu(attrs, outs, target):
+    """schedule pooling ops for x86"""
+    with target:
+        return topi.x86.schedule_pool(outs, attrs.layout)
+
+@schedule_adaptive_pool.register("cpu")
+def schedule_adaptive_pool_cpu(attrs, outs, target):
+    """schedule adaptive pooling ops for x86"""
+    with target:
+        return topi.x86.schedule_adaptive_pool(outs)
+
+@schedule_softmax.register("cpu")
+def schedule_softmax_cpu(attrs, outs, target):
+    """schedule softmax for x86"""
+    with target:
+        return topi.x86.schedule_softmax(outs)
+
+@conv2d_strategy.register("cpu")
+def conv2d_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d x86 strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation_h, dilation_w = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            if topi.x86.is_int8_hw_support(data.dtype, kernel.dtype):
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.conv2d_nchw_int8),
+                    wrap_topi_schedule(topi.x86.schedule_conv2d_nchw_int8))
+            else:
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.conv2d_nchw),
+                    wrap_topi_schedule(topi.x86.schedule_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWIO"
+            logger.warning("For x86 target, NCHW layout is recommended for conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nhwc),
+                wrap_topi_schedule(topi.x86.schedule_conv2d_nhwc))
+        elif layout == "HWCN":
+            assert kernel_layout == "HWIO"
+            logger.warning("For x86 target, NCHW layout is recommended for conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_hwcn),
+                wrap_topi_schedule(topi.generic.schedule_conv2d_hwcn))
+        else:
+            raise RuntimeError("Unsupported conv2d layout {} for cpu".format(layout))
+    elif is_depthwise_conv2d(data.shape, layout, kernel.shape, kernel_layout, groups):
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            channel_multiplier = get_const_tuple(inputs[1].shape)[1]
+            if channel_multiplier == 1:
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.x86.depthwise_conv2d_nchw),
+                    wrap_topi_schedule(topi.x86.schedule_depthwise_conv2d_nchw))
+            else:
+                logger.warning("For x86 target, depthwise_conv2d with channel "
+                               "multiplier greater than 1 is not optimized")
+                strategy.add_implement(
+                    wrap_compute_conv2d(topi.nn.depthwise_conv2d_nchw),
+                    wrap_topi_schedule(topi.generic.schedule_depthwise_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWOI"
+            logger.warning("For x86 target, NCHW layout is recommended for depthwise_conv2d.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nhwc),
+                wrap_topi_schedule(topi.generic.schedule_depthwise_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported depthwise_conv2d layout {}".format(layout))
+    else: # group_conv2d
+        if layout == 'NCHW':
+            assert kernel_layout == "OIHW"
+            logger.warning("group_conv2d is not optimized for cpu.")
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.group_conv2d_nchw, has_groups=True),
+                wrap_topi_schedule(topi.generic.schedule_group_conv2d_nchw))
+        else:
+            raise RuntimeError("Unsupported group_conv2d layout {}".format(layout))
+    return strategy
+
+@conv2d_NCHWc_strategy.register("cpu")
+def conv2d_NCHWc_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d_NCHWc x86 strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    if topi.x86.is_int8_hw_support(data.dtype, kernel.dtype):
+        strategy.add_implement(
+            wrap_compute_conv2d(topi.x86.conv2d_NCHWc_int8, True, True),
+            wrap_topi_schedule(topi.x86.schedule_conv2d_NCHWc_int8))
+    else:
+        strategy.add_implement(
+            wrap_compute_conv2d(topi.x86.conv2d_NCHWc, True, True),
+            wrap_topi_schedule(topi.x86.schedule_conv2d_NCHWc))
+    return strategy
+
+@depthwise_conv2d_NCHWc_strategy.register("cpu")
+def depthwise_conv2d_NCHWc_strategy_cpu(attrs, inputs, out_type, target):
+    """depthwise_conv2d x86 strategy"""
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_compute_conv2d(topi.x86.depthwise_conv2d_NCHWc, True, True),
+        wrap_topi_schedule(topi.x86.schedule_depthwise_conv2d_NCHWc))
+    return strategy
+
+@conv2d_transpose_strategy.register("cpu")
+def conv2d_transpose_strategy_cpu(attrs, inputs, out_type, target):
+    """conv2d_transpose x86 strategy"""
+    layout = attrs.data_layout
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    assert layout == "NCHW", "only support nchw for now"
+    assert dilation == (1, 1), "not support dilate now"
+    assert groups == 1, "only support groups == 1 for now"
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_comptue_conv2d_transpose(topi.x86.conv2d_transpose_nchw),
+        wrap_topi_schedule(topi.x86.schedule_conv2d_transpose_nchw))
+    return strategy
+
+@conv3d_strategy.register("cpu")
+def conv3d_strategy_cpu(attrs, inputs, out_type, target):
+    """conv3d generic strategy"""
+    strategy = _op.OpStrategy()
+    layout = attrs.data_layout
+    if layout == "NCDHW":
+        logger.warning("conv3d with layout NCDHW is not optimized for cpu.")
 
 Review comment:
   That's a good point. But in this specific case, we've already defined the op strategy function for x86. It won't call into generic op strategy function. So the warning message here is still necessary. 
   
   I'll add the message in the generic function when there's no defined strategy function for this target, but only for those complicated ops like conv2d.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379988320
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Name of the implementation */
+  std::string name;
+  /*! \brief Priority level */
+  int plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
 
 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379987954
 
 

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +181,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE = {}
+
+class TopiTemplate(object):
+    """Topi template that holds the topi compute and schedule function"""
+    def __init__(self):
+        self.compute = None
+        self.schedule = None
+        self.customized_func = None
+
+    def __call__(self, *args, **kwargs):
+        args = deserialize_args(args)
+        if self.customized_func is None:
+            return self._default_func(*args, **kwargs)
+        assert callable(self.customized_func)
+        return self.customized_func(*args, **kwargs)
+
+    def _default_func(self, *args, **kwargs):
+        assert callable(self.compute) and callable(self.schedule)
+        out = self.compute(*args, **kwargs)
+        arg_bufs = [out] + self.get_inputs(out)
+        s = self.schedule([out])
+        return s, arg_bufs
+
+    def get_inputs(self, out):
+        inputs = []
+        queue = [out]
+        while queue:
+            t = queue.pop(0)
+            if isinstance(t.op, tensor.PlaceholderOp):
+                inputs.append(t)
+            else:
+                queue.extend(t.op.input_tensors)
+        return inputs
 
-def register(name, func=None, override=False):
-    """Register a task function.
+def register_task_compute(name, func=None):
+    """Register compute function to autotvm task
 
     Parameters
     ----------
-    name : str
-        The name to identify the task.
-    func : callable
-        The function to be registered.
-    override : bool
-        Whether override existing registration.
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
 
     Returns
     -------
-    func: callable
-        The registered function
+    decorator: callable
+        A decorator
     """
-    def _do_reg(myf):
-        if name in TASK_TABLE and not override:
-            raise ValueError(
-                "Key %s is already registered" % name)
-        TASK_TABLE[name] = myf
-        return myf
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.compute is not None:
+            raise ValueError("Compute is already registered in autoTVM task %s" % name)
+        tmpl.compute = f
+        return f
     if func:
         return _do_reg(func)
     return _do_reg
 
-def create(func_name, args, target, target_host=None, template_key=None):
+def register_task_schedule(name, func=None):
+    """Register schedule function to autotvm task
+
 
 Review comment:
   Added and update the 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] yzhliu commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
yzhliu commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380255632
 
 

 ##########
 File path: topi/python/topi/arm_cpu/conv2d_alter_op.py
 ##########
 @@ -0,0 +1,171 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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=invalid-name,unused-variable,unused-argument,no-member
+"""Conv2D alter op and legalize functions for arm cpu"""
+
+import logging
+
+import tvm
+from tvm import relay
+from tvm import autotvm
+
+from ..nn import conv2d_alter_layout
+from ..util import get_const_tuple
+
+
+logger = logging.getLogger('topi')
+
+
+@conv2d_alter_layout.register(["arm_cpu"])
+def _alter_conv2d_layout(attrs, inputs, tinfos, out_type):
+    target = tvm.target.Target.current(allow_none=False)
+    dispatch_ctx = autotvm.task.DispatchContext.current
+
+    _, outs = relay.backend.compile_engine.select_implement(
+        relay.op.get("nn.conv2d"), attrs, tinfos, out_type, target)
+    workload = autotvm.task.get_workload(outs)
+    if workload is None:
+        # The best implementation is not an AutoTVM template,
+        # we then assume it's not necessary to alter this op.
+        return None
+    cfg = dispatch_ctx.query(target, workload)
+    if cfg.is_fallback:  # if is fallback, clear query cache and return None
+        autotvm.task.clear_fallback_cache(target, workload)
 
 Review comment:
   what is in fallback cache?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r381088327
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +75,191 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
+        else:
+            ret.append(dim)
+    return ret
+
+
+def get_valid_implementations(op, attrs, inputs, out_type, target):
+    """Get all valid implementations from the op strategy.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list of tvm.Tensor
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.target.Target
+        The target to compile the op.
+
+    Returns
+    -------
+    ret : list of relay.op.OpImplement
+        The list of op implementations.
+    """
+    fstrategy = op.get_attr("FTVMStrategy")
+    assert fstrategy is not None, "%s doesn't have FTVMStrategy registered" % op.name
+    with target:
+        strategy = fstrategy(attrs, inputs, out_type, target)
+    analyzer = tvm.arith.Analyzer()
+    ret = []
+    for spec in strategy.specializations:
+        if spec.condition:
+            # check if all the clauses in the specialized condition are true
+            flag = True
+            for clause in spec.condition.clauses:
+                clause = analyzer.canonical_simplify(clause)
+                if isinstance(clause, tvm.expr.IntImm) and clause.value:
+                    continue
+                flag = False
+                break
+            if flag:
+                for impl in spec.implementations:
+                    ret.append(impl)
+        else:
+            for impl in spec.implementations:
+                ret.append(impl)
+    return ret
+
+
+def select_implementation(op, attrs, inputs, out_type, target, use_autotvm=True):
+    """Select the best implementation from the op strategy.
+
+    If use_autotvm is True, it'll first try to find the best implementation
+    based on AutoTVM profile results. If no AutoTVM profile result is found,
+    it'll choose the implementation with highest plevel.
+
+    If use_autotvm is False, it'll directly choose the implementation with
+    highest plevel.
+
+    Note that this function doesn't support op with symbolic input shapes.
+
+    Parameters
+    ----------
+    op : relay.op.Op
+        Relay operator.
+
+    attrs : object
+        The op attribute.
+
+    inputs : list[tvm.Tensor]
+        Input tensors to the op.
+
+    out_type : relay.Type
+        The output type.
+
+    target : tvm.target.Target
+        The target to compile the op.
+
+    use_autotvm : bool
+        Whether query AutoTVM to pick the best.
+
+    Returns
+    -------
+    ret : tuple(relay.op.OpImplement, list[tvm.Tensor])
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r379988238
 
 

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +181,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE = {}
+
+class TopiTemplate(object):
+    """Topi template that holds the topi compute and schedule function"""
+    def __init__(self):
+        self.compute = None
+        self.schedule = None
+        self.customized_func = None
+
+    def __call__(self, *args, **kwargs):
+        args = deserialize_args(args)
+        if self.customized_func is None:
+            return self._default_func(*args, **kwargs)
+        assert callable(self.customized_func)
+        return self.customized_func(*args, **kwargs)
+
+    def _default_func(self, *args, **kwargs):
+        assert callable(self.compute) and callable(self.schedule)
+        out = self.compute(*args, **kwargs)
+        arg_bufs = [out] + self.get_inputs(out)
+        s = self.schedule([out])
+        return s, arg_bufs
+
+    def get_inputs(self, out):
+        inputs = []
+        queue = [out]
+        while queue:
+            t = queue.pop(0)
+            if isinstance(t.op, tensor.PlaceholderOp):
+                inputs.append(t)
+            else:
+                queue.extend(t.op.input_tensors)
+        return inputs
 
-def register(name, func=None, override=False):
-    """Register a task function.
+def register_task_compute(name, func=None):
+    """Register compute function to autotvm task
 
     Parameters
     ----------
-    name : str
-        The name to identify the task.
-    func : callable
-        The function to be registered.
-    override : bool
-        Whether override existing registration.
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
 
     Returns
     -------
-    func: callable
-        The registered function
+    decorator: callable
+        A decorator
     """
-    def _do_reg(myf):
-        if name in TASK_TABLE and not override:
-            raise ValueError(
-                "Key %s is already registered" % name)
-        TASK_TABLE[name] = myf
-        return myf
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.compute is not None:
+            raise ValueError("Compute is already registered in autoTVM task %s" % name)
+        tmpl.compute = f
+        return f
     if func:
         return _do_reg(func)
     return _do_reg
 
-def create(func_name, args, target, target_host=None, template_key=None):
+def register_task_schedule(name, func=None):
+    """Register schedule function to autotvm task
+
 
 Review comment:
   This function is not designed to be used by developers. `register_topi_schedule` calls this function to register the schedule function to the task with the task 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#issuecomment-586784823
 
 
   @merrymercy @Hzfengsy would be great if you can help to take a look and comment given this PR Is related to AutoTVM. Let us aim to mege it in the incoming week

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
merrymercy commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r382324425
 
 

 ##########
 File path: python/tvm/autotvm/record.py
 ##########
 @@ -119,20 +124,29 @@ def decode(row, protocol='json'):
 
     Parameters
     ----------
-    row: str
+    row : str
         a row in the logger file
-    protocol: str
+
+    protocol : str
         log protocol, json or pickle
 
     Returns
     -------
-    input: autotvm.tuner.MeasureInput
-    result: autotvm.tuner.MeasureResult
+    ret : tuple(autotvm.tuner.MeasureInput, autotvm.tuner.MeasureResult), or None
+        The tuple of input and result, or None if input uses old version log format.
     """
     # pylint: disable=unused-variable
+    global _old_version_warning
+
     if protocol == 'json':
         row = json.loads(row)
-        tgt, task_name, task_args, task_kwargs, workload, config = row['i']
+        if 'v' in row and row['v'] == 0.1:
+            if _old_version_warning:
+                logger.warning("AutoTVM log version 0.1 is no longer supported.")
 
 Review comment:
   If it is not supported, why don't we just halt here?
   I cannot think of a use case where someone mixes old logs and new logs and wants to ignore the old logs.
   The user should delete all old logs.
   
   Then we can delete the checks like
   ```
   if ret is not None:
       continue
   ```
   in other places.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: src/relay/op/nn/pooling.cc
 ##########
 @@ -164,9 +164,8 @@ bool Pool2DRel(const Array<Type>& types,
 
 template<typename AttrType, topi::nn::PoolType mode>
 Array<te::Tensor> Pool2DCompute(const Attrs& attrs,
-                            const Array<te::Tensor>& inputs,
-                            const Type& out_type,
-                            const Target& target) {
+                                 const Array<te::Tensor>& inputs,
 
 Review comment:
   align

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376057894
 
 

 ##########
 File path: python/tvm/relay/op/strategy/hls.py
 ##########
 @@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Definition of HLS operator strategy."""
+# pylint: disable=invalid-name,unused-argument,wildcard-import,unused-wildcard-import
+from __future__ import absolute_import
+
+import topi
+from .generic import *
+from .. import op as _op
+
+@schedule_injective.register("hls")
+def schedule_injective_hls(attrs, outs, target):
+    """schedule injective ops for hls"""
+    with target:
+        return topi.hls.schedule_injective(outs)
+
+@schedule_reduce.register("hls")
+def schedule_reduce_hls(attrs, outs, target):
+    """schedule reduction ops for hls"""
+    with target:
+        return topi.hls.schedule_reduce(outs)
+
+@schedule_concatenate.register("hls")
+def schedule_concatenate_hls(attrs, outs, target):
+    """schedule concatenate for hls"""
+    with target:
+        return topi.hls.schedule_injective(outs)
+
+@schedule_pool.register("hls")
+def schedule_pool_hls(attrs, outs, target):
+    """schedule pooling ops for hls"""
+    with target:
+        return topi.hls.schedule_pool(outs, attrs.layout)
+
+@schedule_adaptive_pool.register("hls")
+def schedule_adaptive_pool_hls(attrs, outs, target):
+    """schedule adaptive pooling ops for hls"""
+    with target:
+        return topi.hls.schedule_adaptive_pool(outs)
+
+@schedule_softmax.register("hls")
+def schedule_softmax_hls(attrs, outs, target):
+    """schedule softmax for hls"""
+    with target:
+        return topi.hls.schedule_softmax(outs)
+
+@override_native_generic_func("conv2d_strategy")
+def conv2d_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d hls strategy"""
+    strategy = _op.OpStrategy()
+    data, kernel = inputs
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    layout = attrs.data_layout
+    kernel_layout = attrs.kernel_layout
+    (dilation_h, dilation_w) = dilation
+    if dilation_h < 1 or dilation_w < 1:
+        raise ValueError("dilation should be positive value")
+
+    if groups == 1:
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nchw),
+                wrap_topi_schedule(topi.hls.schedule_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWIO"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.conv2d_nhwc),
+                wrap_topi_schedule(topi.hls.schedule_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported conv2d layout {}".format(layout))
+    elif is_depthwise_conv2d(data.shape, layout, kernel.shape, kernel_layout, groups):
+        if layout == "NCHW":
+            assert kernel_layout == "OIHW"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nchw),
+                wrap_topi_schedule(topi.hls.schedule_depthwise_conv2d_nchw))
+        elif layout == "NHWC":
+            assert kernel_layout == "HWOI"
+            strategy.add_implement(
+                wrap_compute_conv2d(topi.nn.depthwise_conv2d_nhwc),
+                wrap_topi_schedule(topi.hls.schedule_depthwise_conv2d_nhwc))
+        else:
+            raise RuntimeError("Unsupported depthwise_conv2d layout {}".format(layout))
+    else: # group_conv2d
+        raise RuntimeError("group_conv2d is not supported for hls")
+    return strategy
+
+@override_native_generic_func("conv2d_NCHWc_strategy")
+def conv2d_NCHWc_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d_NCHWc hls strategy"""
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_compute_conv2d(topi.nn.conv2d_NCHWc, True, True),
+        wrap_topi_schedule(topi.hls.schedule_conv2d_NCHWc))
+    return strategy
+
+@conv2d_transpose_strategy.register("hls")
+def conv2d_transpose_strategy_hls(attrs, inputs, out_type, target):
+    """conv2d_transpose hls strategy"""
+    layout = attrs.data_layout
+    dilation = get_const_tuple(attrs.dilation)
+    groups = attrs.groups
+    assert layout == "NCHW", "only support nchw for now"
+    assert dilation == (1, 1), "not support dilate now"
+    assert groups == 1, "only support groups == 1 for now"
+    strategy = _op.OpStrategy()
+    strategy.add_implement(
+        wrap_comptue_conv2d_transpose(topi.nn.conv2d_transpose_nchw),
 
 Review comment:
   fixed, thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376144115
 
 

 ##########
 File path: include/tvm/relay/op_attr_types.h
 ##########
 @@ -207,13 +216,137 @@ enum AnyCodegenStrategy {
   kVariableDimensions
 };
 
-/* \brief A runtime representation of shape. */
+/*! \brief A runtime representation of shape. */
 using Shape = Array<IndexExpr>;
 
 using FShapeFunc = runtime::TypedPackedFunc<
   Array<te::Tensor>(const Attrs& attrs,
-                     const Array<te::Tensor>& inputs,
-                     const Array<IndexExpr>& out_ndims)>;
+                    const Array<te::Tensor>& inputs,
+                    const Array<IndexExpr>& out_ndims)>;
+
+/*!
+ * \brief Operator implementation in TVM.
+ */
+class OpImplementNode : public Object {
+ public:
+  /*! \brief Compute function */
+  FTVMCompute fcompute;
+  /*! \brief Schedule function */
+  FTVMSchedule fschedule;
+  /*! \brief Name of the implementation */
+  std::string name;
+  /*! \brief Priority level */
+  int plevel;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("name", &name);
+    v->Visit("plevel", &plevel);
+  }
+
+  static constexpr const char* _type_key = "relay.OpImplement";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpImplementNode, Object);
+};
+
+/*!
+ * \brief Operator implementation class.
+ */
+class OpImplement : public ObjectRef {
+ public:
+  /*!
+   * \brief Invoke the operator compute function.
+   * \param attrs The attribute of the primitive
+   * \param inputs The input tensors.
+   * \param out_type The output type information.
+   * \return The output compute description of the operator.
+   */
+  Array<te::Tensor> Compute(const Attrs& attrs,
+                            const Array<te::Tensor>& inputs,
+                            const Type& out_type);
+  /*!
+   * \brief Build the computation schedule.
+   * \param attrs The attribute of the node.
+   * \param outs The output tensors.
+   * \param target The build target.
+   * \return The computation schedule.
+   */
+  te::Schedule Schedule(const Attrs& attrs,
+                        const Array<te::Tensor>& outs,
+                        const Target& target);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(OpImplement, ObjectRef, OpImplementNode);
+};
+
+/*!
+ * \brief Specialized implementations for operators under certain conditions.
+ */
+class OpSpecializationNode : public Object {
+ public:
+  /*! \brief List of implementations. */
+  Array<OpImplement> implements;
+  /*! \brief Condition to enable the specialization.
+   *    Could be undefined to represent generic case. */
+  te::SpecializedCondition condition;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {
+    v->Visit("condition", &condition);
+    v->Visit("implements", &implements);
+  }
+
+  static constexpr const char* _type_key = "relay.OpSpecialization";
+  TVM_DECLARE_FINAL_OBJECT_INFO(OpSpecializationNode, ExprNode);
+};
+
+/*!
+ * \brief Operator specialization class.
+ */
+class OpSpecialization : public ObjectRef {
+ public:
+  /*!
+   * \brief Add an implementation.
+   * \param compute Compute function
 
 Review comment:
   s/compute/fcompute

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r374821520
 
 

 ##########
 File path: python/tvm/relay/backend/compile_engine.py
 ##########
 @@ -63,6 +83,316 @@ def _get_cache_key(source_func, target):
     return source_func
 
 
+def get_shape(shape):
+    """Convert the shape to correct dtype and vars."""
+    ret = []
+    for dim in shape:
+        if isinstance(dim, tvm.expr.IntImm):
+            val = int(dim)
+            assert val <= np.iinfo(np.int32).max
+            ret.append(tvm.expr.IntImm("int32", val))
+        elif isinstance(dim, tvm.expr.Any):
+            ret.append(tvm.var("any_dim", "int32"))
 
 Review comment:
   Each time encounters Any, it will create a new var. These vars are different because they have different pointer. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r376652263
 
 

 ##########
 File path: python/tvm/autotvm/task/space.py
 ##########
 @@ -944,11 +939,10 @@ def from_json_dict(json_dict):
                 raise RuntimeError("Invalid config knob type: " + knob_type)
             entity_map[str(key)] = entity
 
-        return ConfigEntity(index, code_hash, template_key, entity_map, constraints)
+        return ConfigEntity(index, code_hash, entity_map, constraints)
 
     def __repr__(self):
-        return "%s,%s,%s,%d" % (str(self._entity_map)[12:-1], self.template_key,
-                                self.code_hash, self.index)
+        return "%s,%s,%d" % (str(self._entity_map)[12:-1], self.code_hash, self.index)
 
 Review comment:
   Sure. Do you have any proposal for the change?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
icemelon9 commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r381029999
 
 

 ##########
 File path: docs/api/python/dev.rst
 ##########
 @@ -28,9 +28,9 @@ tvm.expr
    :members:
    :undoc-members:
 
-tvm.codegen
-~~~~~~~~~~~
-.. automodule:: tvm.codegen
+tvm.target.codegen
+~~~~~~~~~~~~~~~~~~
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/autotvm/task/task.py
 ##########
 @@ -116,43 +181,134 @@ def __repr__(self):
             self.name, self.args, self.kwargs, self.workload
         )
 
-TASK_TABLE = {
-}
+TASK_TABLE = {}
+
+class TopiTemplate(object):
+    """Topi template that holds the topi compute and schedule function"""
+    def __init__(self):
+        self.compute = None
+        self.schedule = None
+        self.customized_func = None
+
+    def __call__(self, *args, **kwargs):
+        args = deserialize_args(args)
+        if self.customized_func is None:
+            return self._default_func(*args, **kwargs)
+        assert callable(self.customized_func)
+        return self.customized_func(*args, **kwargs)
+
+    def _default_func(self, *args, **kwargs):
+        assert callable(self.compute) and callable(self.schedule)
+        out = self.compute(*args, **kwargs)
+        arg_bufs = [out] + self.get_inputs(out)
+        s = self.schedule([out])
+        return s, arg_bufs
+
+    def get_inputs(self, out):
+        inputs = []
+        queue = [out]
+        while queue:
+            t = queue.pop(0)
+            if isinstance(t.op, tensor.PlaceholderOp):
+                inputs.append(t)
+            else:
+                queue.extend(t.op.input_tensors)
+        return inputs
 
-def register(name, func=None, override=False):
-    """Register a task function.
+def register_task_compute(name, func=None):
+    """Register compute function to autotvm task
 
     Parameters
     ----------
-    name : str
-        The name to identify the task.
-    func : callable
-        The function to be registered.
-    override : bool
-        Whether override existing registration.
+    name: str
+        The task name
+
+    func: None or callable
+        If it is None, return a decorator.
+        If is callable, decorate this function.
 
     Returns
     -------
-    func: callable
-        The registered function
+    decorator: callable
+        A decorator
     """
-    def _do_reg(myf):
-        if name in TASK_TABLE and not override:
-            raise ValueError(
-                "Key %s is already registered" % name)
-        TASK_TABLE[name] = myf
-        return myf
+    def _do_reg(f):
+        if name not in TASK_TABLE:
+            TASK_TABLE[name] = TopiTemplate()
+        tmpl = TASK_TABLE[name]
+        if tmpl.compute is not None:
+            raise ValueError("Compute is already registered in autoTVM task %s" % name)
+        tmpl.compute = f
+        return f
     if func:
         return _do_reg(func)
     return _do_reg
 
-def create(func_name, args, target, target_host=None, template_key=None):
+def register_task_schedule(name, func=None):
+    """Register schedule function to autotvm task
+
 
 Review comment:
   add a code example section 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380342661
 
 

 ##########
 File path: python/tvm/autotvm/record.py
 ##########
 @@ -130,9 +135,17 @@ def decode(row, protocol='json'):
     result: autotvm.tuner.MeasureResult
 
 Review comment:
   Add `Optional` or `None` to the return 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #4644: Relay op strategy

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

 ##########
 File path: python/tvm/relay/memory_alloc.py
 ##########
 @@ -28,8 +28,8 @@
 
 
 def is_primitive(call):
-    return hasattr(call.op, 'attrs') and hasattr(call.op.attrs, 'Primitive') and \
-        int(call.op.attrs.Primitive) == 1
+    return hasattr(call, 'op') and hasattr(call.op, 'attrs') and \
 
 Review comment:
   Can we reuse `is_primitive` defined in `Function`? something like `return hasattr(call, 'op') and isinstance(call.op, expr.Function) and call.op.is_primitive()`? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] comaniac commented on a change in pull request #4644: [WIP] Relay op strategy

Posted by GitBox <gi...@apache.org>.
comaniac commented on a change in pull request #4644: [WIP] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r372693044
 
 

 ##########
 File path: python/tvm/autotvm/task/dispatcher.py
 ##########
 @@ -481,8 +412,12 @@ def _query_inside(self, target, workload):
         """
         if self._counter < len(self._records):
             cfg = self._records[self._counter][0].config
+            wkl = self._records[self._counter][0].task.workload
+            if workload is not None:
+                assert wkl == workload
             self._counter += 1
-            self.update(target, workload, cfg)
+            self.update(target, wkl, cfg)
+            cfg.workload = wkl
 
 Review comment:
   Where is `cfg.workload` initialized? I didn't find a definition in `ConfigSpace`. Also what's the purpose to have a workload field in a config space?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #4644: [Relay][AutoTVM] Relay op strategy
URL: https://github.com/apache/incubator-tvm/pull/4644#discussion_r380376581
 
 

 ##########
 File path: src/relay/op/nn/convolution.h
 ##########
 @@ -153,6 +153,16 @@ bool Conv2DRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
       << " But got " << out_layout;
 
   Array<IndexExpr> dshape_nchw = trans_in_layout.ForwardShape(data->shape);
+  bool is_depthwise = false;
+  if (param->groups > 1) {
+    CHECK(weight && weight->shape.defined()) <<
+        "Weight shape must be specified when groups is greater than 1.";
+    Array<IndexExpr> wshape_oihw = trans_kernel_layout.ForwardShape(weight->shape);
+    if (tvm::tir::Equal(param->groups, dshape_nchw[1]) &&
+        tvm::tir::Equal(param->groups, wshape_oihw[0])) {
 
 Review comment:
   ah right, thanks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services