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 2021/04/07 14:07:58 UTC

[GitHub] [tvm] Hzfengsy opened a new pull request #7806: [TensorIR] introduce pass lower_init_block

Hzfengsy opened a new pull request #7806:
URL: https://github.com/apache/tvm/pull/7806


   This PR is the first part of the stage M1c, TensorIR upstreaming plan (#7527) on lowering init block.
   
   We just use `if` branch to replace the init block. e.g.
   
   Before
   ```Python
   with tir.block([16, 16, tir.reduce_axis(16)]) as [i, j ,k]:
       with tir.init():
           C[i, j] = tir.float32(0)
       C[i, j] += A[i, k] * B[j, k]
   ```
   After
   ```Python
   with tir.block([16, 16, tir.reduce_axis(16)]) as [i, j ,k]:
       if k == 0:
           C[i, j] = tir.float32(0)
       C[i, j] += A[i, k] * B[j, k]
   ```
   
   Please note that we only provide the default behavior for lowering the init part, which is not performent. We will provide primitives for users to manually "decompose" init part.
   
   cc @tqchen @junrushao1994 @jroesch @comaniac @jcf94 
   


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

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



[GitHub] [tvm] comaniac commented on a change in pull request #7806: [TensorIR][PASS] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {

Review comment:
       Could you clarify the meaning of the word "realize" here?




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7806: [TensorIR][PASS] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {
+    std::vector<PrimExpr> conditions;
+    for (const IterVar& var : iter_vars) {
+      if (var->iter_type == IterVarType::kCommReduce) {
+        conditions.push_back(equal(var->var, var->dom->min));
+      }
+    }
+    // Handle the case where there is no condition
+    if (conditions.empty()) {
+      return init;
+    }
+    // Concat the conditions with logical and (&&)
+    PrimExpr cond = conditions[0];
+    for (int i = 1; i < static_cast<int>(conditions.size()); ++i) {

Review comment:
       consider use size_t i here so you do not need to static cast




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

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



[GitHub] [tvm] tqchen merged pull request #7806: [TensorIR][M1c] LowerInitBlock

Posted by GitBox <gi...@apache.org>.
tqchen merged pull request #7806:
URL: https://github.com/apache/tvm/pull/7806


   


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

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



[GitHub] [tvm] comaniac commented on a change in pull request #7806: [TensorIR][M1c] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {

Review comment:
       DoLowering sounds good to me.




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

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



[GitHub] [tvm] comaniac commented on a change in pull request #7806: [TensorIR][M1c] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {

Review comment:
       I got your point, and I feel "LowerInitBlock" itself is the best name to describe what it does, so maybe the simplest way is manually inlining this function so that we won't be bothered by its name, or just give it a general helper name to let people know this is just the implementation of lowering init 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.

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



[GitHub] [tvm] tqchen commented on pull request #7806: [TensorIR][M1c] LowerInitBlock

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


   Thanks @comaniac @junrushao1994 for reviewing. Thanks @Hzfengsy  


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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #7806: [TensorIR][M1c] LowerInitBlock

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



##########
File path: tests/python/unittest/test_tir_transform_lower_init_block.py
##########
@@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import tvm
+from tvm import tir
+from tvm.script import ty
+
+
+@tvm.script.tir
+class WithInit:
+    def main(a: ty.handle, b: ty.handle) -> None:
+        A = tir.match_buffer(a, [64, 64, 64])
+        B = tir.match_buffer(b, [64])
+
+        with tir.block([64, tir.reduce_axis(0, 64), tir.reduce_axis(32, 64)]) as [i, j, k]:
+            with tir.init():
+                B[i] = tir.float32(0)
+            B[i] += A[i, j, k]
+
+
+@tvm.script.tir
+class WithBranch:
+    def main(a: ty.handle, b: ty.handle) -> None:
+        A = tir.match_buffer(a, [64, 64, 64])
+        B = tir.match_buffer(b, [64])
+
+        with tir.block([64, tir.reduce_axis(0, 64), tir.reduce_axis(32, 64)]) as [i, j, k]:
+            if (j == 0) and (k == 32):
+                B[i] = tir.float32(0)
+            B[i] += A[i, j, k]
+
+
+def test_lower_reduction():
+    origin_mod = WithInit()

Review comment:
       Just a note here: per discussion with @Hzfengsy, we need to remove "()" here in future PRs, so that the script parser API is consistent across PrimFunc and IRModule




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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #7806: [TensorIR][M1c] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {
+    std::vector<PrimExpr> conditions;
+    for (const IterVar& var : iter_vars) {
+      if (var->iter_type == IterVarType::kCommReduce) {
+        conditions.push_back(equal(var->var, var->dom->min));
+      }
+    }
+    // Handle the case where there is no condition
+    if (conditions.empty()) {
+      return init;
+    }
+    // Concat the conditions with logical and (&&)
+    PrimExpr cond = conditions[0];
+    for (int i = 1; i < static_cast<int>(conditions.size()); ++i) {

Review comment:
       I am not in favor of using unsigned integers anywhere, in this particular case when things could go either way and we don't have particular reason that unsigned integers are unavoidable, I would prefer using integers instead




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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #7806: [TensorIR][M1c] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {

Review comment:
       This helper is something a bit standalone (making the init block to If-Then-Else), and inlining this function might make the logic in the function above a bit vague, so what about we just call it "DoLowering"?




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

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



[GitHub] [tvm] tqchen commented on a change in pull request #7806: [TensorIR][PASS] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {
+    std::vector<PrimExpr> conditions;
+    for (const IterVar& var : iter_vars) {
+      if (var->iter_type == IterVarType::kCommReduce) {
+        conditions.push_back(equal(var->var, var->dom->min));
+      }
+    }
+    int n = conditions.size();

Review comment:
       seems we can directly refer to conditions.size() here, more readable




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

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



[GitHub] [tvm] junrushao1994 commented on a change in pull request #7806: [TensorIR][M1c] LowerInitBlock

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



##########
File path: src/tir/transforms/lower_init_block.cc
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * Lower block init stmt into branch stmt
+ * \file lower_reduction.cc
+ */
+#include <tvm/tir/op.h>
+#include <tvm/tir/stmt_functor.h>
+#include <tvm/tir/transform.h>
+
+namespace tvm {
+namespace tir {
+
+class InitBlockLower : public StmtMutator {
+ private:
+  Stmt VisitStmt_(const BlockNode* block) final {
+    if (!block->init.defined()) {
+      return StmtMutator::VisitStmt_(block);
+    }
+    Stmt init = RealizeInitBlock(block->init.value(), block->iter_vars);
+    Stmt body = VisitStmt(block->body);
+    auto n = CopyOnWrite(block);
+    n->init = NullOpt;
+    n->body = SeqStmt::Flatten(init, body);
+    return Block(n);
+  }
+
+  static Stmt RealizeInitBlock(const Stmt& init, const Array<IterVar>& iter_vars) {

Review comment:
       What about "concretize"? My understanding is that the init-block is something "conceptual" for initializing "zero point" of a reduction, and this pass is used to make it concrete, lowering to "if-then-else" statement




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

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