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/06/21 01:25:19 UTC

[GitHub] [tvm] ArmageddonKnight opened a new pull request, #11793: [DietCode] Local Padding

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

   This PR is for the code generation changes required for dynamic MetaScheduler (see apache/tvm-rfcs#72 for the RFC, #11516 for the tracking issue describing the changes). Any feedback or comments are welcome.
   
   FYI, @comaniac @junrushao1994 


-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   Hi @junrushao , it has been sometime since this PR is submitted. Any updates on this? Are further changes required?


-- 
This is an automated message from the 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] junrushao1994 commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
tests/python/unittest/test_tir_transform_local_pad.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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
+import numpy as np
+from tvm.script import tir as T
+import tvm.testing
+from tvm.tir import Schedule
+
+
+def sample_dense_sched(sch):  # pylint: disable=too-many-statements

Review Comment:
   Two comments on this unittest:
   
   First, in latest TVM infrastructure, we do not need to do end-to-end testing (i.e. generating cuda kernels, do some approximate equality check, etc) in unittests. Instead, we write TVMScript from bare hand.
   
   Some examples in testing split/fuse in TIR schedule: https://github.com/apache/tvm/blob/main/tests/python/unittest/test_tir_schedule_split_fuse.py
   
   Therefore, to test a single specific pass or two passes (local-pad and vectorize-loop), it's preferred to write two TVMScripts to represent the input of this pass and the corresponding expected outcome, then invoke the specific pass, and finally do structural equality assertion to check correctness.
   
   This way, you will get rid of using anything from TIR Schedule or end-to-end `tvm.build` pipeline, but instead focus on one specific pass as the definition of being "unit" in unittests.
   
   Second comment is about TVMScript. It actually supports limited metaprogrmming right now, so you don't have to write two matmuls:
   
   ```python
   def matmul(M: int, N: int, K: int):
       @T.prim_func
       def kernel(a: T.handle, b: T.handle, c: T.handle) -> None:
           A = T.match_buffer(a, [M, K], dtype="float32")
           B = T.match_buffer(b, [K, N], dtype="float32")
           C = T.match_buffer(c, [M, N], dtype="float32")
           for i, j, k in T.grid(M, N, K):
               with T.block("update"):
                   vi, vj, vk = T.axis.remap("SSR", [i, j, k])
                   with T.init():
                       C[vi, vj] = T.float32(0)
                   C[vi, vj] = C[vi, vj] + A[vi, vk] * B[vj, vk]
   
       return kernel
   
   tir_prim_func = matmul(960, 2304, 770)
   ```
   



-- 
This is an automated message from the 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] junrushao1994 commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/driver/driver_api.cc:
##########
@@ -165,9 +166,10 @@ TVM_REGISTER_GLOBAL("driver.get_binds")
       return out_arr;
     });
 
-Array<tvm::transform::Pass> CreatePassList(bool disable_loop_partition) {
+Array<tvm::transform::Pass> CreatePassList(bool simple_mode) {

Review Comment:
   Thanks for fix the naming :-) Didn't notice it before



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

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

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


[GitHub] [tvm] renfeier commented on pull request #11793: [DietCode] Local Padding

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

   hi, @ArmageddonKnight 
   it seems the tvm transform config "tir.enable_local_pad "does not work since the same schedule build  result kernel src code are the same when set config tir.enable.lcal_pad true/false, when i use the test example you upload before, example code will show belows:
    -------------------------------------------------
    def save_kernel_source(kernel, log_kernel_filename):
        kernel_src=kernel.imported_modules[0].get_source()
        if log_kernel_filename is not None:
            with open(log_kernel_filename, 'w') as fout:
                fout.write("{}".format(kernel_src))
        else:
            print({}.format(kernel_src))
   
    @tvm.testing.requires_gpu
    @tvm.testing.requires_cuda
    def test_dense_local_padding():
        """
        Test that local padding is delivering the correct compute outcome.
        """
        x_np = np.random.uniform(-0.1, 0.1, size=(960, 770)).astype(np.float32)
        w_np = np.random.uniform(-0.1, 0.1, size=(770, 2304)).astype(np.float32)
        y_np = np.matmul(x_np, w_np)
        y_empty = np.empty(shape=y_np.shape, dtype=y_np.dtype)
        tir_sched = Schedule(Dense_960x770x2304)
        sample_dense_sched(tir_sched)
        with tvm.transform.PassContext(config={"tir.enable_local_pad": False}):
            nopad_cuda_kernel = tvm.build(tir_sched.mod["main"], [], target="cuda")
            save_kernel_source(nopad_cuda_kernel, "nolocalpad_kernel.cu")
        with tvm.transform.PassContext(config={"tir.enable_local_pad": True}):
            cuda_kernel = tvm.build(tir_sched.mod["main"], [], target="cuda")
            save_kernel_source(cuda_kernel, "localpad_kernel.cu")
    
        cuda_ctx = tvm.cuda()
        module_data = [x_np, w_np, y_empty]
        module_data = [tvm.nd.array(d, device=cuda_ctx) for d in module_data]
        cuda_kernel(*module_data)
        np.testing.assert_allclose(module_data[-1].numpy(), y_np, atol=1e-3, rtol=1e-3)
   ---------------------------------------------------------------------------------------
   the localpad_kernel.cu are same with nolocalpad_kernel.cu


-- 
This is an automated message from the 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] junrushao1994 commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
tests/python/unittest/test_tir_transform_local_pad.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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
+import numpy as np
+from tvm.script import tir as T
+import tvm.testing
+from tvm.tir import Schedule
+
+
+def sample_dense_sched(sch):  # pylint: disable=too-many-statements

Review Comment:
   Two comments on this unittest:
   
   First, in latest TVM infrastructure, we do not need to do end-to-end testing (i.e. generating cuda kernels, do some approximate equality check, etc) in unittests. Instead, we write TVMScript from bare hand.
   
   Some examples in testing split/fuse in TIR schedule: https://github.com/apache/tvm/blob/main/tests/python/unittest/test_tir_schedule_split_fuse.py
   
   Therefore, to test a single specific pass or two passes (local-pad and vectorize-loop), it's preferred to write two TVMScripts to represent the input of this pass and the corresponding expected outcome, then invoke the specific pass, and finally do structural equality assertion to check correctness.
   
   This way, you will get rid of using anything from TIR Schedule or end-to-end `tvm.build` pipeline, but instead focus on one specific pass as the definition of being "unit" in unittests.
   
   Second comment is about TVMScript. It actually supports limited metaprogrmming right now. For exameple
   



-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };

Review Comment:
   Changes made.



-- 
This is an automated message from the 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] vinx13 commented on pull request #11793: [DietCode] Local Padding

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

   @masahi PadEinsum can achieve something similar when the padding block


-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>

Review Comment:
   Header 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.

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }

Review Comment:
   Actually we do not need to detect opaque accesses, because our target for local padding is shared variables and local variables.



-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @renfeier The reason is ebcause we are refactoring the implementation, so the pass itself is temporarily commented out. Sorry I was quite busy with university business and will finish the refactoring recently.


-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }

Review Comment:
   Opaque accesses in MMA instructions are checked 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.

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

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


[GitHub] [tvm] comaniac commented on pull request #11793: [DietCode] Local Padding

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

   Also cc @Hzfengsy @vinx13 @spectrometerHBH @masahi 


-- 
This is an automated message from the 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] junrushao1994 commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };

Review Comment:
   Instead let's just use enum class like:
   
   ```C++
   enum class StorageType : int32_t {
     kGlobal = 0,
     kShared = 1,
     kLocal = 2,
     kOthers = 3,
   };
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }

Review Comment:
   Use `runtime::StorageScope::Create`: https://github.com/apache/tvm/blob/64c6405d8253fd9381822ecd5b5102cdc7c2e3f8/src/runtime/thread_storage_scope.h#L126
   
   Example: https://github.com/apache/tvm/blob/64c6405d8253fd9381822ecd5b5102cdc7c2e3f8/src/tir/schedule/state.cc#L46
   



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {

Review Comment:
   no need to introduce anonymous namespace, which might make stacktrace less informative 



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \

Review Comment:
   nit: you may use `ContainerType`
   
   ```suggestion
     void VisitExpr_(const OpType::ContainerType* op) final {    \
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");
+    if (StructuralEqual()(op->indices[0], predicate_lhs_)) {
+      predicate_inlinable_ = false;
+    }
+    return StmtVisitor::VisitStmt_(op);
+  }
+  /*!
+   * \brief Check if a predicate can be inlined.
+   */
+  template <typename OpNodeType>
+  bool CanInlinePredicate_(const OpNodeType* op) {
+    predicate_inlinable_ = true;
+    predicate_lhs_ = op->a;
+    VisitStmt(body_stmt_);
+    return predicate_inlinable_;
+  }
+
+  Stmt body_stmt_;
+  std::vector<PrimExpr> inlinable_predicates_, non_inlinable_residuals_;
+  bool predicate_inlinable_;
+  PrimExpr predicate_lhs_;
+
+  friend class LocalPadder;
+};

Review Comment:
   Generally, I would love to propose that we restructure the logic of this class a little bit.
   
   Looks like the analyzer is interested in the following patterns:
   
   ```python
   if A </<=/>=/> X:
     B[Y] = ...
   ```
   
   If so, there isn't much reason to use a visitor pattern because recursion didn't actually happen. Instead, let's go with a more plain and readable fashion, for example
   
   ```C++
   // inputs:
   IfThenElse if_then_else;
   // extract the lhs & rhs of the if-condition
   PrimExpr predicate_lhs{nullptr};
   PrimExpr predicate_rhs{nullptr};
   if (const auto *op = if_then_else->condition.as<LENode>()) {
     predicate_lhs = op->a;
     predicate_rhs = op->a;
   } else if (...) {
     // use a macro or something to deal with LT, GE, GT
   }
   // then let's analyze the body statement
   const BufferStoreNode* buffer_store = if_then_else->then_case.as<BufferStoreNode>();
   ICHECK(buffer_store);
   if (StructuralEqual()(buffer_store->indices[0], predicate_lhs)) {
     ... // some logic here
   } else {
     ... // some logic here
   }
   ```
   



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");
+    if (StructuralEqual()(op->indices[0], predicate_lhs_)) {
+      predicate_inlinable_ = false;
+    }

Review Comment:
   so it means in:
   
   ```python
   if X < 100:  # `predicate_lhs_` is X
     A[X] = ... # `op->indices[0]` is X
   ```
   
   we are not going to inline the predicate?



##########
include/tvm/tir/transform.h:
##########
@@ -117,11 +127,13 @@ TVM_DLL Pass LoopPartition();
 /*!
  * \brief Lower vectorization loops.
  *
- * \param enable_vectorize Whether vectorization is enabled.
+ * \param enable_vectorize  Whether vectorization is enabled.
+ * \param enable_local_pad  Whether local padding is enabled. Local padding can affect
+ *                          how vectorization is made.
  *
  * \return The pass.
  */
-TVM_DLL Pass VectorizeLoop(bool enable_vectorize = true);
+TVM_DLL Pass VectorizeLoop(const bool enable_vectorize = true, const bool enable_local_pad = false);

Review Comment:
   nit: there is no difference (for scalars) to add the `const` specifier
   
   ```suggestion
   TVM_DLL Pass VectorizeLoop(bool enable_vectorize = true, bool enable_local_pad = false);
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {

Review Comment:
   there is no need to specify with extra `const`, because the pointer address is never mutated in this program, and the compiler can usually figure it out in CFA
   
   ```suggestion
       if (const ImmNodeType* rhs_val = rhs.as<ImmNodeType>()) {
         if (init_constexpr_.defined()) {
           if (const ImmNodeType* init_val = init_constexpr_.as<ImmNodeType>()) {
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;

Review Comment:
   Use explicit nullability
   
   ```suggestion
     Optional<PrimExpr> init_constexpr_;
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }

Review Comment:
   Besides `BufferLoad` and `BufferStore`, there is another buffer access pattern called opaque access - accessing a buffer with intrinsics, for example, tensor core's MMA intrinsics.
   
   To check if a buffer `b` is opaque-accessed, we only need to check if `b->data` (whose type is `tir::Var`) is visited in `StmtExprVisitor`.
   
   Example: https://github.com/apache/tvm/blob/f8186d8c7d3e4679a6dfd83d17521f20bfb3ca42/src/tir/schedule/primitive/compute_inline.cc#L223-L226. where we check if `inlined_buffer` is opaquely accessed.



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {

Review Comment:
   nit: No need to add the underscore given it's already a private method
   
   ```suggestion
     void CheckInitValue(const PrimExpr& rhs) {
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");

Review Comment:
   ditto. Use `runtime::StorageScope` instead of bare strings



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \

Review Comment:
   nit: usually when defining macros, we will have to find a longer name in case it conflicts. for example:
   
   ```suggestion
   #define TVM_TIR_TRANSFORM_LOCAL_PAD_VISIT_PREDICATE(OpType)                      \
   ```



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \

Review Comment:
   Quick question: Do we expect to continue visiting `op->a` and `op->b`?



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}

Review Comment:
   Quick question: do we assume that `body_stmt` is a BufferStore?



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {

Review Comment:
   ditto. Usually we put underscore after variables but not methods



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓

Review Comment:
   just nitpicking...can we avoid using non-ascii comment in the code base? i remembered there used to be some weird compilation issues on windows...
   
   ```suggestion
    *            |
   ```



-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {

Review Comment:
   Remove underscore of private methods.



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

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");
+    if (StructuralEqual()(op->indices[0], predicate_lhs_)) {
+      predicate_inlinable_ = false;
+    }

Review Comment:
   Yes.



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

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {

Review Comment:
   Qualifiers 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.

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");
+    if (StructuralEqual()(op->indices[0], predicate_lhs_)) {
+      predicate_inlinable_ = false;
+    }
+    return StmtVisitor::VisitStmt_(op);
+  }
+  /*!
+   * \brief Check if a predicate can be inlined.
+   */
+  template <typename OpNodeType>
+  bool CanInlinePredicate_(const OpNodeType* op) {
+    predicate_inlinable_ = true;
+    predicate_lhs_ = op->a;
+    VisitStmt(body_stmt_);
+    return predicate_inlinable_;
+  }
+
+  Stmt body_stmt_;
+  std::vector<PrimExpr> inlinable_predicates_, non_inlinable_residuals_;
+  bool predicate_inlinable_;
+  PrimExpr predicate_lhs_;
+
+  friend class LocalPadder;
+};

Review Comment:
   I am afraid we cannot directly do it like this. The reason is because the predicates are usually combined together into a single one and hence we need some way of splitting them. The implementation you provide might not be able to handle situations like the following:
   
   ```C++
   if (inlineable_predicate1 && non_inlineable_predicate2 && inlineable_predicate3)
     A_shared[...] = A[...];
   
   |  // after transformation
   
   if (noninlineable_predicate2)
     A_shared[...] = inlineable_predicate1 && inlineable_predicate3 ? A[...] : padded_value;
   ```



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

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }

Review Comment:
   Changes made. `StorageScope::Create` is used 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.

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓

Review Comment:
   Change made.



-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
tests/python/unittest/test_tir_transform_local_pad.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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
+import numpy as np
+from tvm.script import tir as T
+import tvm.testing
+from tvm.tir import Schedule
+
+
+def sample_dense_sched(sch):  # pylint: disable=too-many-statements

Review Comment:
   Changes made. Testing is now done at the TVM script level.



-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   Hmm ... seems that the Cortex CI pipelines are always interrupted for some reason. 


-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @tvm-bot rerun


-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \

Review Comment:
   No. We do not have to continue visiting `op->a/b`. We would like to treat the predicate as a whole and there is no point in further decomposing it.



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

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \

Review Comment:
   Changes made.



##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \

Review Comment:
   Changes made.



-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");

Review Comment:
   Changes made accordingly.



-- 
This is an automated message from the 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] junrushao1994 commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
tests/python/unittest/test_tir_transform_local_pad.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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
+import numpy as np
+from tvm.script import tir as T
+import tvm.testing
+from tvm.tir import Schedule
+
+
+def sample_dense_sched(sch):  # pylint: disable=too-many-statements

Review Comment:
   Two comments on this unittest:
   
   First, in latest TVM infrastructure, we do not need to do end-to-end testing (i.e. generating cuda kernels, do some approximate equality check, etc) in unittests. Instead, we write TVMScript from bare hand.
   
   Some examples in testing split/fuse in TIR schedule: https://github.com/apache/tvm/blob/main/tests/python/unittest/test_tir_schedule_split_fuse.py
   
   Therefore, to test a single specific pass or two passes (local-pad and vectorize-loop), it's preferred to write two TVMScripts to represent the input of this pass and the corresponding expected outcome, then invoke the specific pass, and finally do structural equality assertion to check correctness
   



-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @junrushao1994 As is discussed, I have fixed the implementation. Please review it 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.

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 #11793: [DietCode] Local Padding

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

   This looks similar to https://github.com/apache/tvm/pull/12750, maybe we don't need it? cc @vinx13 


-- 
This is an automated message from the 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] comaniac commented on pull request #11793: [DietCode] Local Padding

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

   Per offline discussion with @junrushao1994 and @ArmageddonKnight, here is the current action items:
   1. The local padding pass will be moved to TIR transformation, meaning that local padding becomes an implicit transformation similar to loop partitioning. A config will be exposed to control whether to turn on or off (default off) to keep all current workloads unchanged.
   2. In the local padding implementation, the logic related to var node name hints will be improved to leverage a more reliable factor (e.g., pointer reference).


-- 
This is an automated message from the 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 #11793: [DietCode] Local Padding

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

   @ArmageddonKnight @junrushao What is the status of this PR or DietCode upstreaming in general? I'm interested in dynamic shape tuning, and I can help this effort. 


-- 
This is an automated message from the 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] renfeier commented on pull request #11793: [DietCode] Local Padding

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

   > refactoring
   @ArmageddonKnight 
   Thank you for the prompt reply. Looking forward  to your update
   


-- 
This is an automated message from the 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] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}
+
+#define VISIT_PREDICATE(OpType)                      \
+  void VisitExpr_(const OpType##Node* op) final {    \
+    OpType predicate = GetRef<OpType>(op);           \
+    if (CanInlinePredicate_<OpType##Node>(op)) {     \
+      inlinable_predicates_.push_back(predicate);    \
+    } else {                                         \
+      non_inlinable_residuals_.push_back(predicate); \
+    }                                                \
+  }
+  VISIT_PREDICATE(LT)
+  VISIT_PREDICATE(LE)
+  VISIT_PREDICATE(GT)
+  VISIT_PREDICATE(GE)
+#undef VISIT_PREDICATE
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    if (op->indices.size() != 1) {
+      return StmtVisitor::VisitStmt_(op);
+    }
+    CHECK(op->buffer.scope() == "shared" || op->buffer.scope() == "local");
+    if (StructuralEqual()(op->indices[0], predicate_lhs_)) {
+      predicate_inlinable_ = false;
+    }
+    return StmtVisitor::VisitStmt_(op);
+  }
+  /*!
+   * \brief Check if a predicate can be inlined.
+   */
+  template <typename OpNodeType>
+  bool CanInlinePredicate_(const OpNodeType* op) {
+    predicate_inlinable_ = true;
+    predicate_lhs_ = op->a;
+    VisitStmt(body_stmt_);
+    return predicate_inlinable_;
+  }
+
+  Stmt body_stmt_;
+  std::vector<PrimExpr> inlinable_predicates_, non_inlinable_residuals_;
+  bool predicate_inlinable_;
+  PrimExpr predicate_lhs_;
+
+  friend class LocalPadder;
+};

Review Comment:
   I am afraid we cannot directly do it like this. The reason is because the predicates are usually combined together into a single one and hence we need some way of splitting them. The implementation you provide might not be able to handle situations like the following:
   
   ```C++
   if (inlineable_predicate1 && non_inlineable_predicate2 && inlineable_predicate3)
     A_shared[...] = A[...];
   
   |  // after transformation
   
   if (noninlineable_predicate2
     A_shared[...] = inlineable_predicate1 && inlineable_predicate3 ? A[...] : padded_value;
   ```



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

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
include/tvm/tir/transform.h:
##########
@@ -117,11 +127,13 @@ TVM_DLL Pass LoopPartition();
 /*!
  * \brief Lower vectorization loops.
  *
- * \param enable_vectorize Whether vectorization is enabled.
+ * \param enable_vectorize  Whether vectorization is enabled.
+ * \param enable_local_pad  Whether local padding is enabled. Local padding can affect
+ *                          how vectorization is made.
  *
  * \return The pass.
  */
-TVM_DLL Pass VectorizeLoop(bool enable_vectorize = true);
+TVM_DLL Pass VectorizeLoop(const bool enable_vectorize = true, const bool enable_local_pad = false);

Review Comment:
   Qualifiers 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.

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;

Review Comment:
   Changes made.



-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @tvm-bot rerun


-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @junrushao1994 @Hzfengsy I have finished the revision. Please have a second look when you have time. 
   
   Also cc @comaniac 


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

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

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


[GitHub] [tvm] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   It seems that for some reason the CI build is stopped, would it be possible to re-trigger the CI.


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

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

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


[GitHub] [tvm] junrushao1994 commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>

Review Comment:
   remove 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.

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

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


[GitHub] [tvm] ArmageddonKnight commented on a diff in pull request #11793: [DietCode] Local Padding

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


##########
src/tir/transforms/local_pad.cc:
##########
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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/postproc.h>
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+#include <array>
+#include <utility>
+#include <vector>
+
+namespace tvm {
+namespace tir {
+namespace transform {
+namespace {
+
+/*!
+ * \brief Analyze the read and write accesses of the body statements, used by `LocalPadder`.
+ */
+class StorageAccessAnalyzer : public StmtExprVisitor {
+ private:
+  struct StorageType {
+    enum { kGlobal = 0, kShared, kLocal, kOthers };
+  };
+
+  void VisitStmt_(const BufferStoreNode* op) final {
+    write_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitStmt_(op);
+  }
+  void VisitExpr_(const BufferLoadNode* op) final {
+    read_marker_.SetStorageAccessMarker_(op->buffer);
+    StmtExprVisitor::VisitExpr_(op);
+  }
+  class AccessMarker {
+   public:
+    void SetStorageAccessMarker_(const Buffer& buf) {
+      if (buf.scope() == "global") {
+        bit_vector_[StorageType::kGlobal] = true;
+      } else if (buf.scope() == "shared") {
+        bit_vector_[StorageType::kShared] = true;
+      } else if (buf.scope() == "local") {
+        bit_vector_[StorageType::kLocal] = true;
+      } else {
+        bit_vector_[StorageType::kOthers] = true;
+      }
+    }
+    bool NoAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kShared] ||
+               bit_vector_[StorageType::kLocal] || bit_vector_[StorageType::kOthers]);
+    }
+    bool OnlyGlobalAccesses() const {
+      return !(bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal] ||
+               bit_vector_[StorageType::kOthers]) &&
+             bit_vector_[StorageType::kGlobal];
+    }
+    bool OnlyLocalOrSharedAccesses() const {
+      return !(bit_vector_[StorageType::kGlobal] || bit_vector_[StorageType::kOthers]) &&
+             (bit_vector_[StorageType::kShared] || bit_vector_[StorageType::kLocal]);
+    }
+
+   private:
+    std::array<bool, StorageType::kOthers + 1> bit_vector_ = {false};
+  };
+  AccessMarker read_marker_, write_marker_;
+  std::pair<AccessMarker, AccessMarker> Analyze_(const Stmt& stmt) {
+    VisitStmt(stmt);
+    return std::make_pair(read_marker_, write_marker_);
+  }
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Verify that all local variables are initialized to the same constant expression.
+ */
+class InitChecker : public StmtVisitor {
+ private:
+  void VisitStmt_(const BufferStoreNode* op) final {
+    // Read the check the RHS values, make sure that they are the same constant for all the
+    // initialization statements.
+    CheckInitValue_<IntImmNode>(op->value);
+    CheckInitValue_<FloatImmNode>(op->value);
+    return StmtVisitor::VisitStmt_(op);
+  }
+  template <typename ImmNodeType>
+  void CheckInitValue_(const PrimExpr& rhs) {
+    if (const ImmNodeType* const rhs_val = rhs.as<ImmNodeType>()) {
+      if (init_constexpr_.defined()) {
+        if (const ImmNodeType* const init_val = init_constexpr_.as<ImmNodeType>()) {
+          if (rhs_val->value != init_val->value) {
+            init_with_single_constexpr_ = false;
+          }
+        } else {
+          init_with_single_constexpr_ = false;
+        }
+      } else {
+        init_with_single_constexpr_ = true;
+        init_constexpr_ = rhs;
+      }
+    }
+  }
+  void operator()(const Stmt& stmt) {
+    StmtVisitor::operator()(stmt);
+    if (!init_with_single_constexpr_) {
+      init_constexpr_ = PrimExpr();
+    }
+  }
+
+  bool init_with_single_constexpr_ = false;
+  PrimExpr init_constexpr_;
+
+  friend class LocalPadder;
+};
+
+/*!
+ * \brief Split a predicate into inlinable and non-inlinable component.
+ *
+ *        We refer to "inlinable predicate" as
+ *
+ *            if (predicate) A = ...;
+ *            ↓
+ *            A = predicate ? ... : init_constexpr;
+ *
+ *        Note that not all predicates can be inlined. For example, if a predicate is there to guard
+ *        against out-of-boundary accesses to local/shared variables, then it cannot be inlined.
+ */
+class PredicateInliner : public StmtExprVisitor {
+ private:
+  explicit PredicateInliner(const Stmt& body_stmt) : body_stmt_(body_stmt) {}

Review Comment:
   Yes. We assume that the body statements where predicates are to be inlined is a `BufferStore` node.



-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @tvm-bot rerun


-- 
This is an automated message from the 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] ArmageddonKnight commented on pull request #11793: [DietCode] Local Padding

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

   @junrushao1994 The refactored implementation has passed the CI tests. Please review it when you have time available. 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.

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

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