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/10/24 13:11:16 UTC

[GitHub] [incubator-tvm] jcf94 opened a new pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   In Ansor, we have an optimization called "Layout rewrite", which modifies the input weight of a specific op according to its schedule to get better performance, a previous PR about this is #6297 .
   
   This PR brings another option for this feature, besides directly modify the input placeholder, we now support to insert a transpose stage between the placeholder and compute op.
   
   Others:
   - Add a CopyOnWrite methods for transform_steps
   - Some small fix
   
   cc @merrymercy @comaniac @minminsun 


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the shapes of input tensor according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transform stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] merrymercy merged pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between these two options is whether we add new stages. So I suggest the following names:
   
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the shapes of input tensor according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transform stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors as inputs.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the layout of input tensors according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transformation stages for input placeholders in the compute DAG */
     InsertTransformStage = 1,
     /*!
      * \brief Do not insert layout transformation stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   ```
   
   
   In addition, I prefer "transform" over "transpose" because we can support other kinds of rewriting besides the current simple "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



[GitHub] [incubator-tvm] FrozenGene commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   > Hey, thank you for the contribution! May I know the difference between layout rewrite and this weight pre-transpose? Looks like weight pre-transpose can be done in compile time, so why do we insert a stage instead? Thanks a lot!
   
   This could keep the weight shape is the same as before. However, I am curious too what benefit this will bring in. 


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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   It turns out that #6828 didn't really disable the flaky tests. It simply comments out the function call in `__main__`, but `pytest` will look for all functions with `test_` as the prefix. I'll file another PR to disable them.


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the layout of input tensors according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transformation stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   #6841 filed.


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the layout of input tensors according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transformation stages for input placeholders in the compute DAG */
     InsertTransformStage = 1,
     /*!
      * \brief Do not insert layout transformation stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };```
   
   In addition, I prefer "transform" over "transpose" because we can support other kinds of rewritings besides the current simple "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



[GitHub] [incubator-tvm] jcf94 commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   @junrushao1994 @FrozenGene Thanks!
   Our former implementation of layout rewrite is used for end to end relay integration, and we'll modify the input tensor directly since all these can be seen as parts of the pre-process.
   Currently we got a simpler demand, we would like to get the well scheduled kernel from TVM and apply it to some where else. We want to keep the input/output size the same, so it can be used as a out-of-the-box kernel, while at the same time we can still benefit from the layout rewrite optimization.


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the layout of input tensors according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transform stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the layout of input tensors according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transformation stages for input placeholders in the compute DAG */
     InsertTransformStage = 1,
     /*!
      * \brief Do not insert layout transformation stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   In addition, I prefer "transform" over "transpose" because we can support other kinds of rewritings besides the current simple "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



[GitHub] [incubator-tvm] jcf94 edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   @junrushao1994 @FrozenGene Thanks!
   Our former implementation of layout rewrite is used for end to end relay integration, and we'll modify the input tensor directly since all these can be seen as parts of the pre-process.
   Currently we got a simpler demand, we would like to get the well scheduled kernel from TVM and apply it to some where else(out of TVM runtime). We want to keep the input/output size the same, so it can be used as a out-of-the-box kernel, while at the same time we can still benefit from the layout rewrite optimization.


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

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



[GitHub] [incubator-tvm] merrymercy commented on a change in pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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



##########
File path: python/tvm/auto_scheduler/compute_dag.py
##########
@@ -50,6 +50,11 @@ class ComputeDAG(Object):
     compute : Union[List[Tensor], str, Schedule]
         Input/output tensors or workload key for a compute declaration.
     """
+    LAYOUT_REWRITE_TABLE = {
+        "NoRewrite": 0,
+        "RewriteWithPlaceholder": 1,
+        "RewriteWithPreTranspose": 2,
+    }

Review comment:
       Why do we use strings? Can we just use variables like this?
   https://github.com/apache/incubator-tvm/blob/8d56164107ac2be9f2f53f13de202a7bbf1c1c66/python/tvm/auto_scheduler/measure.py#L464-L472




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

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



[GitHub] [incubator-tvm] comaniac commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   @merrymercy @jcf94 the tests added by this PR seem flaky. Please see https://ci.tlcpack.ai/blue/organizations/jenkins/tvm/detail/main/124/pipeline


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

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



[GitHub] [incubator-tvm] FrozenGene commented on a change in pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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



##########
File path: include/tvm/auto_scheduler/compute_dag.h
##########
@@ -194,6 +194,22 @@ class ComputeDAGNode : public Object {
   TVM_DECLARE_FINAL_OBJECT_INFO(ComputeDAGNode, Object);
 };
 
+/*!
+ * \brief Several options for applying layout rewrite.
+ * This is a optimization to rewrite the shape of input tensor according to the schedule we get.
+ */
+enum class LayoutRewriteOption : int {

Review comment:
       `enum class LayoutRewriteOption : uint8` should be enough.




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

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



[GitHub] [incubator-tvm] merrymercy commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   @jcf94 Please fix the test cases.


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

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



[GitHub] [incubator-tvm] comaniac commented on a change in pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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



##########
File path: python/tvm/auto_scheduler/compute_dag.py
##########
@@ -50,6 +50,11 @@ class ComputeDAG(Object):
     compute : Union[List[Tensor], str, Schedule]
         Input/output tensors or workload key for a compute declaration.
     """
+    LAYOUT_REWRITE_TABLE = {
+        "NoRewrite": 0,
+        "RewriteWithPlaceholder": 1,
+        "RewriteWithPreTranspose": 2,
+    }

Review comment:
       Is that possible to avoid duplicated layout rewrite option tables, although to be honest I have no idea how to do that...

##########
File path: src/auto_scheduler/compute_dag.cc
##########
@@ -902,28 +903,91 @@ void ComputeDAG::RewriteLayout(const Array<Step>& transform_steps) {
       if (!direct_consumer) {
         continue;
       }
+      handled_ops.insert(placeholder_op);
 
+      // Process original layout
       std::set<std::string> placeholder_axis_names;
-      GetOrigLayout(&placeholder_axis_names, op, placeholder);
+      std::string origin_layout = GetOrigLayout(&placeholder_axis_names, op, placeholder);
+      Array<PrimExpr> origin_shape;
+      std::vector<std::string> origin_axes;
+      ParseKernelLayout(origin_layout, &origin_shape, &origin_axes);
 
-      Array<PrimExpr> new_shape;
+      // Process new layout
       std::string new_layout =
-          GetNewLayout(&new_shape, state, stage_id, stage, op, placeholder, placeholder_axis_names);
-
-      handled_ops.insert(placeholder_op);
-
-      Array<te::Operation> old_ops = p_dag->ops;
-      ArrayNode* pops = p_dag->ops.CopyOnWrite();
-
-      // Create new placeholder
-      te::Operation new_placeholder_op;
-      new_placeholder_op = te::PlaceholderOp(placeholder_op->name, new_shape,
+          GetNewLayout(state, stage_id, stage, op, placeholder, placeholder_axis_names);
+      Array<PrimExpr> new_shape;
+      std::vector<std::string> new_axes;
+      ParseKernelLayout(new_layout, &new_shape, &new_axes);
+
+      // Process op updates
+      te::Operation new_op_to_update;
+      if (layout_rewrite == LayoutRewriteOption::RewriteWithPlaceholder) {
+        // Create new placeholder
+        new_op_to_update = te::PlaceholderOp(placeholder_op->name, new_shape,
                                              placeholder_op.as<te::PlaceholderOpNode>()->dtype);
+      } else if (layout_rewrite == LayoutRewriteOption::RewriteWithPreTranspose) {
+        // Process index strides
+        std::unordered_map<std::string, PrimExpr> axes_stride;
+        for (const auto& i : origin_axes) {
+          axes_stride[i] = Integer(1);
+        }
+        Array<PrimExpr> new_stride(new_shape.size(), PrimExpr());
+        PrimExpr temp = Integer(1);
+        for (int i = new_shape.size() - 1; i >= 0; i--) {
+          new_stride.Set(i, axes_stride[new_axes[i]]);
+          axes_stride[new_axes[i]] *= new_shape[i];
+        }
+
+        // Add extra layout transpose stage
+        const auto& layout_transform_tensor = te::compute(
+            new_shape,
+            [&new_stride, &placeholder_op, &origin_shape, &new_shape, &origin_axes,
+             &new_axes](const tvm::runtime::Array<tvm::tir::Var>& indices) -> tvm::PrimExpr {
+              Array<PrimExpr> access_indices;
+              for (size_t indice_index = 0; indice_index < origin_shape.size(); indice_index++) {
+                PrimExpr temp = Integer(0);
+                for (size_t i = 0; i < new_shape.size(); i++) {
+                  if (origin_axes[indice_index].compare(new_axes[i]) == 0) {
+                    temp += indices[i] * new_stride[i];
+                  }
+                }
+                access_indices.push_back(temp);
+              }
+              return placeholder_op.output(0)(access_indices);
+            },
+            "auto_schedule_layout_transpose");
+        new_op_to_update = layout_transform_tensor->op;
+
+        // Update the transform steps
+        for (size_t i = 0; i < transform_steps->size(); i++) {
+          Step step = (*transform_steps)[i];
+          if (step->stage_id >= static_cast<int>(stage_id)) {
+            step.CopyOnWrite()->stage_id++;
+          }
+          if (step->IsInstance<ComputeAtStepNode>()) {
+            auto compute_at_step = tvm::Downcast<ComputeAtStep>(step);
+            if (compute_at_step->target_stage_id >= static_cast<int>(stage_id)) {
+              dynamic_cast<ComputeAtStepNode*>(compute_at_step.CopyOnWrite())->target_stage_id++;
+            }
+            transform_steps->Set(i, std::move(compute_at_step));
+          } else {
+            transform_steps->Set(i, std::move(step));
+          }
+        }
+        Array<Integer> to_fuse;
+        for (size_t i = 0; i < new_shape.size() - 1; i++) {
+          to_fuse.push_back(i);
+        }
+        transform_steps->push_back(FuseStep(stage_id, to_fuse));
+        transform_steps->push_back(AnnotationStep(stage_id, 0, IteratorAnnotation::kParallel));
+      } else {
+        LOG(FATAL) << "Call ComputeDAG::RewriteLayout with NoRewrite.";

Review comment:
       Better to be an assertion as this is enforced in `ApplySteps`.

##########
File path: include/tvm/auto_scheduler/transform_step.h
##########
@@ -182,7 +182,23 @@ class StepNode : public Object {
  */
 class Step : public ObjectRef {
  public:
-  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(Step, ObjectRef, StepNode);
+  /*!
+   * \brief CopyOnWrite function for Step.
+   * This works almost the same as a normal ObjectRef.CopyOnWrite(), but can dispatch to different
+   * steps.
+   * \return A base StepNode pointer, need to cast to its real StepNode type before doing any
+   * modifies.

Review comment:
       ```suggestion
      * modifications.
   ```

##########
File path: tests/python/unittest/test_auto_scheduler_layout_rewrite.py
##########
@@ -50,16 +55,18 @@ def test_layout_rewrite_correctness():
 
         search_policy = auto_scheduler.SketchPolicy(task)
 
+        measure_ctx = auto_scheduler.LocalRPCMeasureContext()

Review comment:
       Better to `del measure_ctx` explicitly. 

##########
File path: tests/python/unittest/test_auto_scheduler_layout_rewrite.py
##########
@@ -100,10 +107,56 @@ def test_layout_rewrite_correctness():
         func_ref(*args_ref)
         ctx.sync()
 
-        np.testing.assert_allclose(np_args[0], np_args_ref[0])
-        np.testing.assert_allclose(np_args[2], np_args_ref[2])
+        np.testing.assert_allclose(args[0].asnumpy(), args_ref[0].asnumpy())
+        np.testing.assert_allclose(args[2].asnumpy(), args_ref[2].asnumpy())
+
+
+def test_correctness_layout_rewrite_with_pre_transpose():
+    N = 128
+    target = tvm.target.Target("llvm")
+    task = auto_scheduler.create_task(matmul_auto_scheduler_test, (N, N, N), target)
+    dag = task.compute_dag
+
+    with tempfile.NamedTemporaryFile() as fp:
+        log_file = fp.name
+
+        search_policy = auto_scheduler.SketchPolicy(task)
+
+        measure_ctx = auto_scheduler.LocalRPCMeasureContext()

Review comment:
       ditto




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

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



[GitHub] [incubator-tvm] merrymercy commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. I suggest the following names:
   
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the shapes of input tensor according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transform stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the shapes of input tensor according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transform stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. For example, the "RewriteWithPlaceholder" option in your code actually accept "PreTransposed" tensors.
   The key difference between the two options is whether adding new stages. We can make the names much clearer as follows.
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the layout of input tensors according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transformation stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transformation stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   In addition, I prefer "transform" over "transpose" because we can support other kinds of rewritings besides the current simple "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



[GitHub] [incubator-tvm] junrushao1994 commented on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   Hey, thank you for the contribution! May I know the difference between layout rewrite and this weight pre-transpose? Looks like weight pre-transpose can be done in compile time, so why do we insert a stage instead? Thanks a lot!


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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


   The code looks good to me. But the naming is not intuitive. I suggest the following names:
   
   ```c++
   /*!
    * \brief Options for applying layout rewrite.
    * This is an optimization to rewrite the shapes of input tensor according to the schedule we get.
    */
   enum class LayoutRewriteOption : int {
     /*! \brief Do not process layout rewrite. */
     NoRewrite = 0,
     /*! \brief Insert layout transform stages for input placeholders in the compute DAG */
     AddTransformStage = 1,
     /*!
      * \brief Do not insert layout transform stages and assume the input placeholders
      * are pre-transformed.
      * \note The lowered function with this option does not accept the origial input shapes,
      * so this option must be used along with a layout conversion pass in Relay.
      */
     RewriteForPreTransformed = 2,
   };
   
   ```


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

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



[GitHub] [incubator-tvm] merrymercy edited a comment on pull request #6750: [AutoScheduler] New layout rewrite option: Weight pre-transpose

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






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

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