You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2022/11/19 23:51:35 UTC

[GitHub] [tvm] nverke opened a new pull request, #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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

   … that utilize async dma pipelines.
   
   This mostly copies what is happening in MultiLevelTilingTensorCore but alters the pipelining strategy for hexagon async dma pipelining. 
   
   I kept the intrin groups in so that this can be expanded if we want to include a synchronous dma tensorization. 


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

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

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


[GitHub] [tvm] masahi commented on pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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

   I made a new branch based on this PR, to apply `MultiLevelTilingHexagon` in practice: https://github.com/apache/tvm/compare/main...masahi:hex-ms-async?expand=1
   
   Currently, tuning can run but the output is incorrect if async DMA is enabled. If we enable only VTCM cache read and synchronous software pipeline, the output is correct. See the test case in `test_hexagon/test_conv2d_async.py`. Here is an example of a generated IR after `LowerAsyncDMA`: https://gist.github.com/masahi/f15806cb19f5c00da10a12341aef137e
   
   The main changes from this PR are:
   * Start from `MultiLevelTilingWithIntrin`, rather than `MultiLevelTilingTensorCore`, to avoid unnecessary complications like `reindex` and `transform_layout`.
   *  Using `AddReadReuse` with level `1` leads to `cache_read` block being placed under a spatial loop. The pipeline annotation is also attached to the wrong loop, which makes `DecomposeReduction` fail to place the initialization block outside of the pipeline loop. This results in an error `pipeline_stages.size() != original_order.size() (4 vs 3)` (the init block is counted as one of the blocks in the pipeline loop). Using `levels = [2]` fixes these issues.
   * Moreover, rather than pipelining only over a single reduction loop, fuse all reduction loops above a cache-read block. For example, in
   ```
   for i0_0, i1_0, i2_0, i3_0, i4_0_0, i5_0, i6_0, i7_0, i8_0, i9_0_0 in T.grid(T.int64(1), T.int64(2), T.int64(6), T.int64(3), T.int64(1), T.int64(1), T.int64(3), T.int64(1), T.int64(4), T.int64(1)):
       for ax0_ax1_ax2_ax3_ax4_fused in T.serial(T.int64(3168)):
           with T.block("p0_global.vtcm"):
              ...
       for ax0_ax1_ax2_ax3_ax4_ax5_ax6_fused in T.serial(T.int64(1536)):
           with T.block("fused_constant_global.vtcm"):
              ...
   ```
    `i5_0, i6_0, i7_0, i8_0, i9_0_0` are all reduction loops, which can be fused to make a bigger pipelined loop. After fusing and applying the annotations, it looks like:
   ```
   for i0_0, i1_0, i2_0, i3_0, i4_0_0 in T.grid(T.int64(1), T.int64(1), T.int64(1), T.int64(1), T.int64(1)):
       for i5_0_i6_0_i7_0_i8_0_i9_0_0_fused in T.serial(T.int64(12), annotations={"software_pipeline_async_stages":[0], "software_pipeline_order":[0, 1, 2], "software_pipeline_stage":[0, 0, 1]}):
           for ax0_ax1_ax2_ax3_ax4_fused in T.serial(T.int64(48384)):
               with T.block("p0_global.vtcm"):
                 ...
           for ax0_ax1_ax2_ax3_ax4_ax5_ax6_fused in T.serial(T.int64(3072)):
               with T.block("fused_constant_global.vtcm"):
                 ...
   ```
   * Disable parallelization, since doing `dma_copy` under a parallel loop is not supported by the current Hexagon runtime.   


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

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

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


[GitHub] [tvm] nverke commented on a diff in pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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


##########
tests/python/unittest/test_meta_schedule_schedule_rule_mlt_hexagon.py:
##########
@@ -0,0 +1,973 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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=missing-module-docstring,missing-function-docstring,missing-class-docstring
+from tests.python.contrib.test_hexagon.test_meta_schedule import dense_compute
+import tvm
+from tvm.meta_schedule import schedule_rule
+import tvm.testing
+from tvm import meta_schedule as ms
+from tvm import te
+from tvm.meta_schedule.testing import te_workload
+from tvm.meta_schedule.testing.space_generation import ( 
+    check_sketches,
+    generate_design_space,
+    get_rules,
+)
+from tvm.script import tir as T
+from tvm.tir.tensor_intrin.cuda import get_wmma_intrin_group
+from tvm.tir.tensor_intrin.hexagon import VRMPY_u8u8i32_INTRIN, VRMPY_u8u8i32_VTCM_INTRIN
+
+
+def multi_level_tiling_hexagon(
+    *,
+    write_reuse_scope="global.vtcm",
+    in_dtype="uint8",
+    out_dtype="int32",
+    use_software_pipeline=False,
+) -> ms.schedule_rule.ScheduleRule:
+    assert write_reuse_scope in ["global", "global.vtcm"]
+    if not isinstance(in_dtype, list):
+        in_dtype = [in_dtype]
+    if not isinstance(out_dtype, list):
+        out_dtype = [out_dtype]
+    return ms.schedule_rule.MultiLevelTilingHexagon(
+        intrin_groups=[
+            {"compute": VRMPY_u8u8i32_VTCM_INTRIN},
+        ],
+        structure="SRSRS",
+        tile_binds=None,
+        max_innermost_factor=64,  # 64 // tensor intrin size
+        vector_load_lens=None,
+        reuse_read=ms.schedule_rule.ReuseType(
+            req="must",
+            levels=[1],

Review Comment:
   Ahh I see that now I will make changes to the test!



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

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

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


[GitHub] [tvm] nverke commented on pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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

   Awesome! Ill take a look at that branch! 


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

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

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


[GitHub] [tvm] nverke commented on a diff in pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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


##########
src/meta_schedule/schedule_rule/multi_level_tiling_hexagon.cc:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include <tvm/meta_schedule/schedule_rule.h>
+
+#include <algorithm>
+#include <utility>
+#include <vector>
+
+#include "../utils.h"
+#include "./multi_level_tiling.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+using tir::BlockRV;
+using tir::LoopRV;
+using tir::Schedule;
+
+struct HexagonIntrinGroup {
+  String compute_intrin;
+
+  /*! \brief Create HexagonIntrinGroup from config in a map. The map should contains the
+   * following keys:
+   *  - compute
+   * The values of the keys should be the names of the corresponding intrinsics and should be
+   * registered via TensorIntrin.Register beforehand.
+   */
+  static HexagonIntrinGroup FromConfig(const Map<String, String>& config);
+};
+
+HexagonIntrinGroup HexagonIntrinGroup::FromConfig(const Map<String, String>& config) {
+  auto f_initialize_intrin = [&config](String key_name, String* intrin_name) {
+    CHECK(config.count(key_name)) << "ValueError: " << key_name << " is not set.";
+    *intrin_name = config.at(key_name);
+    // Check the existence of the intrin
+    tir::TensorIntrin::Get(*intrin_name);
+  };
+  HexagonIntrinGroup intrin_group;
+  f_initialize_intrin("compute", &intrin_group.compute_intrin);
+  return intrin_group;
+}
+
+class HexagonStateNode : public StateNode {
+ public:
+  /*! \brief The hexagon intrinsic group. */
+  HexagonIntrinGroup intrin_group;
+  /*! \brief The auto tensorization maping info. */
+  tir::AutoTensorizeMappingInfo mapping_info{nullptr};
+  /*! \brief The hexagon reindex block A for hexagon computation */
+  tir::BlockRV hexagon_reindex_A;
+  /*! \brief The hexagon reindex block B for hexagon computation */
+  tir::BlockRV hexagon_reindex_B;
+  /*! \brief The hexagon reindex store block for hexagon computation */
+  tir::BlockRV hexagon_reindex_store;
+
+  State Copy() const final;
+
+  static constexpr const char* _type_key = "meta_schedule.TensorCoreState";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HexagonStateNode, StateNode);
+};
+
+class HexagonState : public State {
+ public:
+  explicit HexagonState(HexagonIntrinGroup intrin_group, tir::AutoTensorizeMappingInfo mapping_info, Schedule sch, BlockRV block_rv, Array<Array<tir::LoopRV>> tiles = {});
+
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(HexagonState, State, HexagonStateNode);
+};
+
+TVM_REGISTER_OBJECT_TYPE(HexagonStateNode);
+
+HexagonState::HexagonState(HexagonIntrinGroup intrin_group,
+                                 tir::AutoTensorizeMappingInfo mapping_info, Schedule sch,
+                                 BlockRV block_rv, Array<Array<LoopRV>> tiles) {
+  ObjectPtr<HexagonStateNode> node = make_object<HexagonStateNode>();
+  node->intrin_group = intrin_group;
+  node->mapping_info = mapping_info;
+  node->sch = std::move(sch);
+  node->block_rv = std::move(block_rv);
+  node->tiles = std::move(tiles);
+  data_ = std::move(node);
+}
+
+State HexagonStateNode::Copy() const {
+  ObjectPtr<HexagonStateNode> node = make_object<HexagonStateNode>(*this);
+  node->sch = sch->Copy();
+  return State(node);
+}
+
+/*!
+ * \brief Extension of MultiLevelTiling for auto-tensorizing with a single group of hexagon
+ * intrinsics.
+ */
+class MultiLevelTilingHexagonNode : public MultiLevelTilingNode {
+ private:
+  // SubRule: Add tensorization-related transformations
+  inline std::vector<State> TransformForTensorization(HexagonState state) const;
+  // Subrule: Add software pipeline
+  inline std::vector<State> AddSoftwarePipeline(HexagonState state) const;
+
+  // Override ApplySubRules to apply tensorization-specific sub-rules
+  std::vector<State> ApplySubRules(std::vector<State> states) final;
+
+  // Override Apply to apply tensorization-specific analysis before applying sub-rules
+  Array<Schedule> Apply(const Schedule& sch, const BlockRV& block_rv) final;
+
+  // Inherited from ScheduleRuleNode
+  ScheduleRule Clone() const final {
+    ObjectPtr<MultiLevelTilingHexagonNode> n =
+        make_object<MultiLevelTilingHexagonNode>(*this);
+    return ScheduleRule(n);
+  }
+
+  /*!
+   * \brief Transform and tensorize with the given tensor intrin
+   * \param state The state of the meta schedule rule
+   * \param intrin_name The name of the tensor intrin
+   * \return The loop to be tensorized. NullOpt if the workload can't be tensorized.
+   */
+  Optional<LoopRV> TransformWithTensorIntrin(HexagonStateNode* state,
+                                             const String& intrin_name) const;
+
+  /*!
+   * \brief Tile, blockize and annotate for tensorization with the given intrin
+   * \param block_rv The block to be tensorized
+   * \param intrin_name The name of the tensor intrin
+   */
+  void TileAndAnnotateTensorize(Schedule* sch, const BlockRV& block_rv,
+                                const String& intrin_name) const;
+
+ public:
+  /*! \brief The candidate hexagon intrin groups to apply */
+  std::vector<HexagonIntrinGroup> intrin_groups;
+  /*! \brief Whether to use software pipeline */
+  bool use_software_pipeline = false;
+  static constexpr const char* _type_key = "meta_schedule.MultiLevelTilingHexagon";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MultiLevelTilingHexagonNode, MultiLevelTilingNode);
+
+ private:
+};
+
+// Entry of the mega rule; Inherited from ScheduleRuleNode
+Array<Schedule> MultiLevelTilingHexagonNode::Apply(const Schedule& sch, const BlockRV& block_rv) {
+  if (!NeedsMultiLevelTiling(sch->state(), sch->GetSRef(block_rv))) {
+    return {sch};
+  }
+
+  std::unordered_map<int, tir::AutoTensorizeMappingInfo> intrin_group_to_mapping_info;
+  for (int i = 0, n = intrin_groups.size(); i < n; ++i) {
+    HexagonIntrinGroup intrin_group = intrin_groups[i];
+    Optional<tir::AutoTensorizeMappingInfo> mapping_info = tir::GetAutoTensorizeMappingInfo(
+        sch->state(), sch->GetSRef(block_rv),
+        tir::TensorIntrin::Get(intrin_groups[i].compute_intrin).value()->desc);
+    if (mapping_info.defined()) {
+      intrin_group_to_mapping_info.emplace(i, mapping_info.value());
+    }
+  }
+
+  if (intrin_group_to_mapping_info.empty()) {
+    // No tensor intrinsics can be applied.
+    return {sch};
+  }
+
+  // Save the original schedule so that we can roll back transformations if tensorization
+  // fails.
+  Schedule original_sch = sch;
+
+  std::vector<State> initial_states;
+  for (const auto& kv : intrin_group_to_mapping_info) {
+    const HexagonIntrinGroup& intrin_group = intrin_groups[kv.first];
+    const tir::AutoTensorizeMappingInfo& mapping_info = kv.second;
+    Schedule new_sch = sch->Copy();
+    new_sch->Annotate(block_rv, tir::attr::meta_schedule_tiling_structure, structure);
+    initial_states.push_back(HexagonState(intrin_group, mapping_info, new_sch, block_rv));
+  }
+  Array<Schedule> results;
+  for (auto&& state : ApplySubRules(initial_states)) {
+    TVM_PY_LOG(INFO, logger) << "Sketch " << results.size() << ": tensorizing with "
+                             << state.as<HexagonStateNode>()->intrin_group.compute_intrin;
+    results.push_back(std::move(state->sch));
+  }
+  if (results.empty()) {
+    return {original_sch};
+  }
+  return results;
+}
+
+std::vector<State> MultiLevelTilingHexagonNode::ApplySubRules(std::vector<State> states) {
+  states = SubRule(std::move(states), [&](State state) {
+    return TransformForTensorization(Downcast<HexagonState>(state));
+  });
+  states = SubRule(std::move(states), [&](State state) { return TileLoopNest(state); });
+  states = SubRule(std::move(states), [&](State state) { return AddWriteReuse(state); });
+  states = SubRule(std::move(states), [&](State state) { return AddReadReuse(state); });
+  states = SubRule(std::move(states), [&](State state) {
+    return AddSoftwarePipeline(Downcast<HexagonState>(state));
+  });
+  return states;
+}
+
+void MultiLevelTilingHexagonNode::TileAndAnnotateTensorize(Schedule* sch,
+                                                              const BlockRV& block_rv,
+                                                              const String& intrin_name) const {
+  Optional<LoopRV> loop = TileWithTensorIntrin(*sch, block_rv, intrin_name).value();
+  ICHECK(loop.defined());
+  BlockRV blockized_outer = (*sch)->Blockize(loop.value());
+  (*sch)->Annotate(blockized_outer, tir::attr::meta_schedule_auto_tensorize, intrin_name);
+}
+
+std::vector<State> MultiLevelTilingHexagonNode::AddSoftwarePipeline(
+    HexagonState state) const {
+  if (!use_software_pipeline) {
+    return {state};
+  }
+  // The current config is not suitable for software pipelining.
+  if (r_indices_.size() < 2) {
+    return {state};
+  }
+
+  Schedule& sch = state->sch;
+  // Check reduction length after blockize.
+  int64_t reduction_length = 1;
+  for (int r_index : r_indices_) {
+    const Array<LoopRV>& tiles = state->tiles[r_index];
+    for (const LoopRV& tile : tiles) {
+      const auto* extent = sch->Get(tile)->extent.as<IntImmNode>();
+      ICHECK(extent != nullptr) << "Dynamic extent is not supported.";
+      reduction_length *= extent->value;
+    }
+  }
+  if (reduction_length <= 1) {
+    return {state};
+  }
+
+  // Return if there are more less than 1 or more than 2 cache_reads.
+  size_t cache_read_count = state->read_reuse.size();
+  if (cache_read_count > 2 || cache_read_count == 0) {
+    return {state};
+  } 
+  
+  // Add annotations for software pipelining at the loop right above the cache read stages. 
+  tir::BlockRV cache_read_block = state->read_reuse.begin()->second;
+  Array<LoopRV> cache_read_loops = sch->GetLoops(cache_read_block);
+  Array<Integer> software_pipeline_stage; 
+  Array<Integer> software_pipeline_order; 
+  Array<Integer> software_pipeline_async_stages; 
+  if(cache_read_count == 2) {
+    software_pipeline_stage = Array<Integer>{0, 0, 1};
+    software_pipeline_order = Array<Integer>{0, 1, 2};
+    software_pipeline_async_stages = Array<Integer>{0};
+  } else {
+    software_pipeline_stage = Array<Integer>{0, 1};
+    software_pipeline_order = Array<Integer>{0, 1};
+    software_pipeline_async_stages = Array<Integer>{0};
+  }
+  sch->Annotate(cache_read_loops[cache_read_loops.size() - 2], tir::attr::software_pipeline_stage, software_pipeline_stage);

Review Comment:
   Agreed, this is just the way that I have been creating these for testing purposes. We have seen experimentally that there are better configurations. 



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

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

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


[GitHub] [tvm] nverke closed pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

Posted by GitBox <gi...@apache.org>.
nverke closed pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…
URL: https://github.com/apache/tvm/pull/13446


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

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

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


[GitHub] [tvm] tvm-bot commented on pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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

   <!---bot-comment-->
   
   Thanks for contributing to TVM! Please refer to the contributing guidelines https://tvm.apache.org/docs/contribute/ for useful information and tips. Please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @-ing them in a comment.
   
   
   
   <sub>Generated by [tvm-bot](https://github.com/apache/tvm/blob/main/ci/README.md#github-actions)</sub>


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

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

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


[GitHub] [tvm] masahi commented on a diff in pull request #13446: [MetaSchedule] Add MultiLevelTilingHexagon to create simple schedules…

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


##########
src/meta_schedule/schedule_rule/multi_level_tiling_hexagon.cc:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include <tvm/meta_schedule/schedule_rule.h>
+
+#include <algorithm>
+#include <utility>
+#include <vector>
+
+#include "../utils.h"
+#include "./multi_level_tiling.h"
+
+namespace tvm {
+namespace meta_schedule {
+
+using tir::BlockRV;
+using tir::LoopRV;
+using tir::Schedule;
+
+struct HexagonIntrinGroup {
+  String compute_intrin;
+
+  /*! \brief Create HexagonIntrinGroup from config in a map. The map should contains the
+   * following keys:
+   *  - compute
+   * The values of the keys should be the names of the corresponding intrinsics and should be
+   * registered via TensorIntrin.Register beforehand.
+   */
+  static HexagonIntrinGroup FromConfig(const Map<String, String>& config);
+};
+
+HexagonIntrinGroup HexagonIntrinGroup::FromConfig(const Map<String, String>& config) {
+  auto f_initialize_intrin = [&config](String key_name, String* intrin_name) {
+    CHECK(config.count(key_name)) << "ValueError: " << key_name << " is not set.";
+    *intrin_name = config.at(key_name);
+    // Check the existence of the intrin
+    tir::TensorIntrin::Get(*intrin_name);
+  };
+  HexagonIntrinGroup intrin_group;
+  f_initialize_intrin("compute", &intrin_group.compute_intrin);
+  return intrin_group;
+}
+
+class HexagonStateNode : public StateNode {
+ public:
+  /*! \brief The hexagon intrinsic group. */
+  HexagonIntrinGroup intrin_group;
+  /*! \brief The auto tensorization maping info. */
+  tir::AutoTensorizeMappingInfo mapping_info{nullptr};
+  /*! \brief The hexagon reindex block A for hexagon computation */
+  tir::BlockRV hexagon_reindex_A;
+  /*! \brief The hexagon reindex block B for hexagon computation */
+  tir::BlockRV hexagon_reindex_B;
+  /*! \brief The hexagon reindex store block for hexagon computation */
+  tir::BlockRV hexagon_reindex_store;
+
+  State Copy() const final;
+
+  static constexpr const char* _type_key = "meta_schedule.TensorCoreState";
+  TVM_DECLARE_FINAL_OBJECT_INFO(HexagonStateNode, StateNode);
+};
+
+class HexagonState : public State {
+ public:
+  explicit HexagonState(HexagonIntrinGroup intrin_group, tir::AutoTensorizeMappingInfo mapping_info, Schedule sch, BlockRV block_rv, Array<Array<tir::LoopRV>> tiles = {});
+
+  TVM_DEFINE_MUTABLE_OBJECT_REF_METHODS(HexagonState, State, HexagonStateNode);
+};
+
+TVM_REGISTER_OBJECT_TYPE(HexagonStateNode);
+
+HexagonState::HexagonState(HexagonIntrinGroup intrin_group,
+                                 tir::AutoTensorizeMappingInfo mapping_info, Schedule sch,
+                                 BlockRV block_rv, Array<Array<LoopRV>> tiles) {
+  ObjectPtr<HexagonStateNode> node = make_object<HexagonStateNode>();
+  node->intrin_group = intrin_group;
+  node->mapping_info = mapping_info;
+  node->sch = std::move(sch);
+  node->block_rv = std::move(block_rv);
+  node->tiles = std::move(tiles);
+  data_ = std::move(node);
+}
+
+State HexagonStateNode::Copy() const {
+  ObjectPtr<HexagonStateNode> node = make_object<HexagonStateNode>(*this);
+  node->sch = sch->Copy();
+  return State(node);
+}
+
+/*!
+ * \brief Extension of MultiLevelTiling for auto-tensorizing with a single group of hexagon
+ * intrinsics.
+ */
+class MultiLevelTilingHexagonNode : public MultiLevelTilingNode {
+ private:
+  // SubRule: Add tensorization-related transformations
+  inline std::vector<State> TransformForTensorization(HexagonState state) const;
+  // Subrule: Add software pipeline
+  inline std::vector<State> AddSoftwarePipeline(HexagonState state) const;
+
+  // Override ApplySubRules to apply tensorization-specific sub-rules
+  std::vector<State> ApplySubRules(std::vector<State> states) final;
+
+  // Override Apply to apply tensorization-specific analysis before applying sub-rules
+  Array<Schedule> Apply(const Schedule& sch, const BlockRV& block_rv) final;
+
+  // Inherited from ScheduleRuleNode
+  ScheduleRule Clone() const final {
+    ObjectPtr<MultiLevelTilingHexagonNode> n =
+        make_object<MultiLevelTilingHexagonNode>(*this);
+    return ScheduleRule(n);
+  }
+
+  /*!
+   * \brief Transform and tensorize with the given tensor intrin
+   * \param state The state of the meta schedule rule
+   * \param intrin_name The name of the tensor intrin
+   * \return The loop to be tensorized. NullOpt if the workload can't be tensorized.
+   */
+  Optional<LoopRV> TransformWithTensorIntrin(HexagonStateNode* state,
+                                             const String& intrin_name) const;
+
+  /*!
+   * \brief Tile, blockize and annotate for tensorization with the given intrin
+   * \param block_rv The block to be tensorized
+   * \param intrin_name The name of the tensor intrin
+   */
+  void TileAndAnnotateTensorize(Schedule* sch, const BlockRV& block_rv,
+                                const String& intrin_name) const;
+
+ public:
+  /*! \brief The candidate hexagon intrin groups to apply */
+  std::vector<HexagonIntrinGroup> intrin_groups;
+  /*! \brief Whether to use software pipeline */
+  bool use_software_pipeline = false;
+  static constexpr const char* _type_key = "meta_schedule.MultiLevelTilingHexagon";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MultiLevelTilingHexagonNode, MultiLevelTilingNode);
+
+ private:
+};
+
+// Entry of the mega rule; Inherited from ScheduleRuleNode
+Array<Schedule> MultiLevelTilingHexagonNode::Apply(const Schedule& sch, const BlockRV& block_rv) {
+  if (!NeedsMultiLevelTiling(sch->state(), sch->GetSRef(block_rv))) {
+    return {sch};
+  }
+
+  std::unordered_map<int, tir::AutoTensorizeMappingInfo> intrin_group_to_mapping_info;
+  for (int i = 0, n = intrin_groups.size(); i < n; ++i) {
+    HexagonIntrinGroup intrin_group = intrin_groups[i];
+    Optional<tir::AutoTensorizeMappingInfo> mapping_info = tir::GetAutoTensorizeMappingInfo(
+        sch->state(), sch->GetSRef(block_rv),
+        tir::TensorIntrin::Get(intrin_groups[i].compute_intrin).value()->desc);
+    if (mapping_info.defined()) {
+      intrin_group_to_mapping_info.emplace(i, mapping_info.value());
+    }
+  }
+
+  if (intrin_group_to_mapping_info.empty()) {
+    // No tensor intrinsics can be applied.
+    return {sch};
+  }
+
+  // Save the original schedule so that we can roll back transformations if tensorization
+  // fails.
+  Schedule original_sch = sch;
+
+  std::vector<State> initial_states;
+  for (const auto& kv : intrin_group_to_mapping_info) {
+    const HexagonIntrinGroup& intrin_group = intrin_groups[kv.first];
+    const tir::AutoTensorizeMappingInfo& mapping_info = kv.second;
+    Schedule new_sch = sch->Copy();
+    new_sch->Annotate(block_rv, tir::attr::meta_schedule_tiling_structure, structure);
+    initial_states.push_back(HexagonState(intrin_group, mapping_info, new_sch, block_rv));
+  }
+  Array<Schedule> results;
+  for (auto&& state : ApplySubRules(initial_states)) {
+    TVM_PY_LOG(INFO, logger) << "Sketch " << results.size() << ": tensorizing with "
+                             << state.as<HexagonStateNode>()->intrin_group.compute_intrin;
+    results.push_back(std::move(state->sch));
+  }
+  if (results.empty()) {
+    return {original_sch};
+  }
+  return results;
+}
+
+std::vector<State> MultiLevelTilingHexagonNode::ApplySubRules(std::vector<State> states) {
+  states = SubRule(std::move(states), [&](State state) {
+    return TransformForTensorization(Downcast<HexagonState>(state));
+  });
+  states = SubRule(std::move(states), [&](State state) { return TileLoopNest(state); });
+  states = SubRule(std::move(states), [&](State state) { return AddWriteReuse(state); });
+  states = SubRule(std::move(states), [&](State state) { return AddReadReuse(state); });
+  states = SubRule(std::move(states), [&](State state) {
+    return AddSoftwarePipeline(Downcast<HexagonState>(state));
+  });
+  return states;
+}
+
+void MultiLevelTilingHexagonNode::TileAndAnnotateTensorize(Schedule* sch,
+                                                              const BlockRV& block_rv,
+                                                              const String& intrin_name) const {
+  Optional<LoopRV> loop = TileWithTensorIntrin(*sch, block_rv, intrin_name).value();
+  ICHECK(loop.defined());
+  BlockRV blockized_outer = (*sch)->Blockize(loop.value());
+  (*sch)->Annotate(blockized_outer, tir::attr::meta_schedule_auto_tensorize, intrin_name);
+}
+
+std::vector<State> MultiLevelTilingHexagonNode::AddSoftwarePipeline(
+    HexagonState state) const {
+  if (!use_software_pipeline) {
+    return {state};
+  }
+  // The current config is not suitable for software pipelining.
+  if (r_indices_.size() < 2) {
+    return {state};
+  }
+
+  Schedule& sch = state->sch;
+  // Check reduction length after blockize.
+  int64_t reduction_length = 1;
+  for (int r_index : r_indices_) {
+    const Array<LoopRV>& tiles = state->tiles[r_index];
+    for (const LoopRV& tile : tiles) {
+      const auto* extent = sch->Get(tile)->extent.as<IntImmNode>();
+      ICHECK(extent != nullptr) << "Dynamic extent is not supported.";
+      reduction_length *= extent->value;
+    }
+  }
+  if (reduction_length <= 1) {
+    return {state};
+  }
+
+  // Return if there are more less than 1 or more than 2 cache_reads.
+  size_t cache_read_count = state->read_reuse.size();
+  if (cache_read_count > 2 || cache_read_count == 0) {
+    return {state};
+  } 
+  
+  // Add annotations for software pipelining at the loop right above the cache read stages. 
+  tir::BlockRV cache_read_block = state->read_reuse.begin()->second;
+  Array<LoopRV> cache_read_loops = sch->GetLoops(cache_read_block);
+  Array<Integer> software_pipeline_stage; 
+  Array<Integer> software_pipeline_order; 
+  Array<Integer> software_pipeline_async_stages; 
+  if(cache_read_count == 2) {
+    software_pipeline_stage = Array<Integer>{0, 0, 1};
+    software_pipeline_order = Array<Integer>{0, 1, 2};
+    software_pipeline_async_stages = Array<Integer>{0};
+  } else {
+    software_pipeline_stage = Array<Integer>{0, 1};
+    software_pipeline_order = Array<Integer>{0, 1};
+    software_pipeline_async_stages = Array<Integer>{0};
+  }
+  sch->Annotate(cache_read_loops[cache_read_loops.size() - 2], tir::attr::software_pipeline_stage, software_pipeline_stage);

Review Comment:
   `cache_read_loops[cache_read_loops.size() - 2` might not be the correct reduction loop we want to pipeline. Moreover, due to many splitting, a single loop typically has a small extent. So we should fuse all reduction loops above a `cache_read` block. 



##########
tests/python/unittest/test_meta_schedule_schedule_rule_mlt_hexagon.py:
##########
@@ -0,0 +1,973 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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=missing-module-docstring,missing-function-docstring,missing-class-docstring
+from tests.python.contrib.test_hexagon.test_meta_schedule import dense_compute
+import tvm
+from tvm.meta_schedule import schedule_rule
+import tvm.testing
+from tvm import meta_schedule as ms
+from tvm import te
+from tvm.meta_schedule.testing import te_workload
+from tvm.meta_schedule.testing.space_generation import ( 
+    check_sketches,
+    generate_design_space,
+    get_rules,
+)
+from tvm.script import tir as T
+from tvm.tir.tensor_intrin.cuda import get_wmma_intrin_group
+from tvm.tir.tensor_intrin.hexagon import VRMPY_u8u8i32_INTRIN, VRMPY_u8u8i32_VTCM_INTRIN
+
+
+def multi_level_tiling_hexagon(
+    *,
+    write_reuse_scope="global.vtcm",
+    in_dtype="uint8",
+    out_dtype="int32",
+    use_software_pipeline=False,
+) -> ms.schedule_rule.ScheduleRule:
+    assert write_reuse_scope in ["global", "global.vtcm"]
+    if not isinstance(in_dtype, list):
+        in_dtype = [in_dtype]
+    if not isinstance(out_dtype, list):
+        out_dtype = [out_dtype]
+    return ms.schedule_rule.MultiLevelTilingHexagon(
+        intrin_groups=[
+            {"compute": VRMPY_u8u8i32_VTCM_INTRIN},
+        ],
+        structure="SRSRS",
+        tile_binds=None,
+        max_innermost_factor=64,  # 64 // tensor intrin size
+        vector_load_lens=None,
+        reuse_read=ms.schedule_rule.ReuseType(
+            req="must",
+            levels=[1],

Review Comment:
   It seems 2 is the correct level. If we try to use it in practice, the level 1 would lead to the annotation being applied to a non-reduction loop. 



-- 
This is an automated message from the 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