You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2020/07/29 07:56:31 UTC

[GitHub] [incubator-tvm] eric-haibin-lin opened a new pull request #6152: [WIP] Basic block normal form

eric-haibin-lin opened a new pull request #6152:
URL: https://github.com/apache/incubator-tvm/pull/6152


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


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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #6152: [WIP] Basic block normal form

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



##########
File path: src/relay/transforms/to_a_normal_form.cc
##########
@@ -50,10 +50,17 @@ Scope LCA(Scope lhs, Scope rhs) {
   return lhs;
 }
 
-std::unordered_map<DependencyGraph::Node*, Scope> CalcScope(const DependencyGraph& dg) {
-  std::unordered_map<DependencyGraph::Node*, Scope> expr_scope;
+// return a set of Exprs whose scope should be lifted to due dependencies.
+void CalcScope(const DependencyGraph& dg,
+               std::unordered_map<DependencyGraph::Node*, Scope>* expr_scope,

Review comment:
       Why this change?




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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on pull request #6152: [RELAY] Basic block normal form

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


   We can allow it if needed!


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

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



[GitHub] [incubator-tvm] tqchen commented on pull request #6152: [WIP] Basic block normal form

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


   cc @MarisaKirisame @mbrookhart @electriclilies @jroesch would be great if you can help to take a look  at the PR


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

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



[GitHub] [incubator-tvm] junrushao1994 commented on a change in pull request #6152: [WIP] Basic block normal form

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



##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+/* Fill expressions based on each scope's let list. Different from FillANF,
+ * only expressions with lifted scope will be pushed to the let list.
+ */
+class FillBasicBlock : ExprFunctor<Expr(const Expr&, const Var&)> {
+  using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+ public:
+  static Expr ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                     std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                                     ExprSet* lifted) {
+    FillBasicBlock fi(dg, node_scope, lifted);
+    auto var = fi.VisitExpr(e);
+    auto scope = fi.GetScope(e);
+    auto ret = scope->ll->Get(var);

Review comment:
       nit: use less auto, unless the type can be seen in the same line

##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+/* Fill expressions based on each scope's let list. Different from FillANF,
+ * only expressions with lifted scope will be pushed to the let list.
+ */
+class FillBasicBlock : ExprFunctor<Expr(const Expr&, const Var&)> {
+  using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+ public:
+  static Expr ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                     std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                                     ExprSet* lifted) {
+    FillBasicBlock fi(dg, node_scope, lifted);
+    auto var = fi.VisitExpr(e);
+    auto scope = fi.GetScope(e);
+    auto ret = scope->ll->Get(var);
+    return ret;
+  }
+
+ private:
+  const DependencyGraph& dg_;
+  std::unordered_map<DependencyGraph::Node*, Scope>* node_scope_;
+  std::unordered_map<Expr, Expr, ObjectPtrHash, ObjectPtrEqual> memo;
+  ExprSet* lifted_;
+
+  FillBasicBlock(const DependencyGraph& dg,
+                 std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                 std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>* lifted)
+      : dg_(dg), node_scope_(node_scope), lifted_(lifted) {}
+
+  Scope GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
+
+  Scope GetSubScope(const Expr& e, size_t i) {
+    DependencyGraph::Node* n = dg_.expr_node.at(e);
+    auto h = n->children.head;
+    while (i != 0) {
+      CHECK(h);

Review comment:
       maybe add more error msg?

##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+/* Fill expressions based on each scope's let list. Different from FillANF,
+ * only expressions with lifted scope will be pushed to the let list.
+ */
+class FillBasicBlock : ExprFunctor<Expr(const Expr&, const Var&)> {
+  using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+ public:
+  static Expr ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                     std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                                     ExprSet* lifted) {
+    FillBasicBlock fi(dg, node_scope, lifted);
+    auto var = fi.VisitExpr(e);
+    auto scope = fi.GetScope(e);
+    auto ret = scope->ll->Get(var);
+    return ret;
+  }
+
+ private:
+  const DependencyGraph& dg_;
+  std::unordered_map<DependencyGraph::Node*, Scope>* node_scope_;
+  std::unordered_map<Expr, Expr, ObjectPtrHash, ObjectPtrEqual> memo;
+  ExprSet* lifted_;
+
+  FillBasicBlock(const DependencyGraph& dg,
+                 std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                 std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>* lifted)
+      : dg_(dg), node_scope_(node_scope), lifted_(lifted) {}
+
+  Scope GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
+
+  Scope GetSubScope(const Expr& e, size_t i) {
+    DependencyGraph::Node* n = dg_.expr_node.at(e);
+    auto h = n->children.head;
+    while (i != 0) {
+      CHECK(h);
+      --i;
+      h = h->next;
+    }
+    CHECK(h);
+    return node_scope_->at(h->value);
+  }
+
+  Expr VisitExpr(const Expr& e, const Var& v) final {
+    if (memo.count(e) == 0) {
+      memo.insert({e, ExprFunctor<Expr(const Expr&, const Var&)>::VisitExpr(e, v)});
+    } else if (v.defined()) {
+      GetScope(e)->ll->Push(v, memo.at(e));
+    }
+    auto ret = memo.at(e);
+    return ret;
+  }
+
+  Expr VisitExpr(const Expr& e) { return this->VisitExpr(e, Var()); }
+
+  Expr Atomic(const Expr& e, const Var& v) { return v.defined() ? GetScope(e)->ll->Push(v, e) : e; }
+
+  // Bind expression `now` to var `v` if the original expression's scope should be lifted, or
+  // if v is defined (e.g. coming from a Let expression). Otherwise return `now` directly.
+  Expr Compound(const Expr& orig, const Expr& now, const Var& v) {
+    Var var = v.defined() ? v : Var(String("x"), Type());
+    if (v.defined() || lifted_->find(orig) != lifted_->end()) {
+      return GetScope(orig)->ll->Push(var, now);
+    } else {
+      return now;
+    }
+  }
+
+  Expr VisitExpr_(const CallNode* c, const Var& v) final {
+    Expr e = GetRef<Expr>(c);
+    std::vector<Expr> args;
+    for (const auto& a : c->args) {
+      args.push_back(VisitExpr(a));
+    }
+    return Compound(e, Call(VisitExpr(c->op), args, c->attrs, c->type_args), v);
+  }
+
+  Expr VisitExpr_(const TupleNode* t, const Var& v) final {
+    Expr e = GetRef<Expr>(t);
+    std::vector<Expr> fields;
+    for (const auto& a : t->fields) {
+      fields.push_back(VisitExpr(a));
+    }
+    return Compound(e, Tuple(fields), v);
+  }
+
+  Expr VisitExpr_(const TupleGetItemNode* t, const Var& v) final {
+    Expr e = GetRef<Expr>(t);
+    return Compound(e, TupleGetItem(VisitExpr(t->tuple), t->index), v);
+  }
+
+  Expr VisitExpr_(const RefCreateNode* r, const Var& v) final {
+    Expr e = GetRef<Expr>(r);
+    return Compound(e, RefCreate(VisitExpr(r->value)), v);
+  }
+
+  Expr VisitExpr_(const RefReadNode* r, const Var& v) final {
+    Expr e = GetRef<Expr>(r);
+    return Compound(e, RefRead(VisitExpr(r->ref)), v);
+  }
+
+  Expr VisitExpr_(const RefWriteNode* r, const Var& v) final {
+    Expr e = GetRef<Expr>(r);
+    return Compound(e, RefWrite(VisitExpr(r->ref), VisitExpr(r->value)), v);
+  }
+
+  Expr VisitExpr_(const IfNode* i, const Var& v) final {
+    Expr e = GetRef<Expr>(i);
+    Expr ret = If(VisitExpr(i->cond), GetSubScope(e, 1)->ll->Get(VisitExpr(i->true_branch)),
+                  GetSubScope(e, 2)->ll->Get(VisitExpr(i->false_branch)));
+    return Compound(e, ret, v);
+  }
+
+  Expr VisitExpr_(const FunctionNode* f, const Var& v) final {
+    Expr e = GetRef<Expr>(f);
+    Expr ret;
+    if (f->HasNonzeroAttr(attr::kPrimitive)) {
+      ret = e;
+    } else {
+      ret = Function(f->params, GetSubScope(e, 0)->ll->Get(VisitExpr(f->body)), f->ret_type,
+                     f->type_params, f->attrs);
+    }
+    return Compound(e, ret, v);
+  }
+
+  Expr VisitExpr_(const LetNode* l, const Var& v) final {
+    Expr e = GetRef<Expr>(l);
+    VisitExpr(l->value, l->var);
+    Expr ret = GetSubScope(e, 0)->ll->Get(VisitExpr(l->body));
+    return Compound(e, ret, v);
+  }
+
+  Expr VisitExpr_(const ConstantNode* c, const Var& v) final {
+    Expr e = GetRef<Expr>(c);
+    return Compound(e, e, v);
+  }
+
+  Expr VisitExpr_(const VarNode* vn, const Var& v) final {
+    Expr e = GetRef<Expr>(vn);
+    return Atomic(e, v);
+  }
+
+  Expr VisitExpr_(const GlobalVarNode* gvn, const Var& v) final {
+    GlobalVar gv = GetRef<GlobalVar>(gvn);
+    return Atomic(gv, v);
+  }
+
+  Expr VisitExpr_(const OpNode* op, const Var& v) final {
+    Expr e = GetRef<Expr>(op);
+    return Atomic(e, v);
+  }
+
+  Expr VisitExpr_(const ConstructorNode* c, const Var& v) final {
+    Expr e = GetRef<Expr>(c);
+    return Atomic(e, v);
+  }
+
+  Expr VisitExpr_(const MatchNode* m, const Var& v) final {
+    Expr e = GetRef<Expr>(m);
+    Expr data = VisitExpr(m->data);
+    std::vector<Clause> clauses;
+    for (const Clause& c : m->clauses) {
+      clauses.push_back(
+          Clause(c->lhs, GetSubScope(e, 1 + clauses.size())->ll->Get(VisitExpr(c->rhs))));
+    }
+    return Compound(e, Match(data, clauses, m->complete), v);
+  }
+};
+
+Expr ToBasicBlockNormalFormAux(const Expr& e) {
+  // calculate all the dependency between nodes.
+  support::Arena arena;
+  DependencyGraph dg = DependencyGraph::Create(&arena, e);
+  /* The scope of the whole expr is global.
+   * The scope of any subexpr, is the lowest common ancestor of all incoming edge.
+   * We also record the set of expressions whose scope is lifted.
+   */
+  std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual> lifted;
+  std::unordered_map<DependencyGraph::Node*, Scope> node_scope;
+  CalcScope(dg, &node_scope, &lifted);
+  return FillBasicBlock::ToBasicBlockNormalForm(e, dg, &node_scope, &lifted);
+}
+
+IRModule ToBasicBlockNormalForm(const IRModule& mod) {
+  DLOG(INFO) << "ToBBlock:" << std::endl << mod;
+
+  tvm::Map<GlobalVar, Function> updates;
+  auto funcs = mod->functions;
+  for (const auto& it : funcs) {
+    CHECK_EQ(FreeVars(it.second).size(), 0);

Review comment:
       Add some error message

##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+/* Fill expressions based on each scope's let list. Different from FillANF,
+ * only expressions with lifted scope will be pushed to the let list.
+ */
+class FillBasicBlock : ExprFunctor<Expr(const Expr&, const Var&)> {
+  using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+ public:
+  static Expr ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                     std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                                     ExprSet* lifted) {
+    FillBasicBlock fi(dg, node_scope, lifted);
+    auto var = fi.VisitExpr(e);
+    auto scope = fi.GetScope(e);
+    auto ret = scope->ll->Get(var);
+    return ret;
+  }
+
+ private:
+  const DependencyGraph& dg_;
+  std::unordered_map<DependencyGraph::Node*, Scope>* node_scope_;
+  std::unordered_map<Expr, Expr, ObjectPtrHash, ObjectPtrEqual> memo;
+  ExprSet* lifted_;
+
+  FillBasicBlock(const DependencyGraph& dg,
+                 std::unordered_map<DependencyGraph::Node*, Scope>* node_scope,
+                 std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>* lifted)
+      : dg_(dg), node_scope_(node_scope), lifted_(lifted) {}
+
+  Scope GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
+
+  Scope GetSubScope(const Expr& e, size_t i) {
+    DependencyGraph::Node* n = dg_.expr_node.at(e);
+    auto h = n->children.head;
+    while (i != 0) {
+      CHECK(h);
+      --i;
+      h = h->next;
+    }
+    CHECK(h);
+    return node_scope_->at(h->value);
+  }
+
+  Expr VisitExpr(const Expr& e, const Var& v) final {
+    if (memo.count(e) == 0) {
+      memo.insert({e, ExprFunctor<Expr(const Expr&, const Var&)>::VisitExpr(e, v)});
+    } else if (v.defined()) {
+      GetScope(e)->ll->Push(v, memo.at(e));
+    }
+    auto ret = memo.at(e);
+    return ret;

Review comment:
       nit
   ```suggestion
       return memo.at(e);
   ```




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

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



[GitHub] [incubator-tvm] mbrookhart commented on a change in pull request #6152: [RELAY] Basic block normal form

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



##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+Expr ToBasicBlockNormalFormAux(const Expr& e) {

Review comment:
       :) I missed this naming influence in the conversations @tqchen and I had a few months ago. Thanks! That makes things a lot clearer for 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] [incubator-tvm] junrushao1994 commented on pull request #6152: [WIP] Basic block normal form

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


   Ping me if the PR is ready to review :-)


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

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



[GitHub] [incubator-tvm] MarisaKirisame merged pull request #6152: [RELAY] Basic block normal form

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


   


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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on pull request #6152: [RELAY] Basic block normal form

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


   Thanks @eric-haibin-lin @mbrookhart @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.

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



[GitHub] [incubator-tvm] eric-haibin-lin commented on pull request #6152: Basic block normal form

Posted by GitBox <gi...@apache.org>.
eric-haibin-lin commented on pull request #6152:
URL: https://github.com/apache/incubator-tvm/pull/6152#issuecomment-667185829


   I added a short description in the PR summary and an analysis pass `check_basic_block_normal_form`. The PR is ready for review now, thanks! 


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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #6152: Basic block normal form

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



##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +188,37 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();
+  explicit ScopeNode(const Scope& parent) : level(1 + parent->level), parent(parent) {}
+  ScopeNode() : level(0) {}
+};
+
+/*! \brief Calculate the scope of nodes in the dependency graph by least common ancestor.
+ *
+ *  \param dg the input dependency graph
+ *  \param expr_scope the output node -> scope mapping for all nodes.
+ *  \param lifted_exprs the output set of expressions whose scope is lifted due to dependency
+ */
+void CalcScope(const DependencyGraph& dg,
+               std::unordered_map<DependencyGraph::Node*, Scope>* expr_scope,

Review comment:
       why use pointer instead of returning?




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

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



[GitHub] [incubator-tvm] eric-haibin-lin commented on a change in pull request #6152: [RELAY] Basic block normal form

Posted by GitBox <gi...@apache.org>.
eric-haibin-lin commented on a change in pull request #6152:
URL: https://github.com/apache/incubator-tvm/pull/6152#discussion_r464588608



##########
File path: src/relay/transforms/to_a_normal_form.cc
##########
@@ -81,171 +70,187 @@ std::unordered_map<DependencyGraph::Node*, Scope> CalcScope(const DependencyGrap
       global_scope_used = true;
     } else {
       s = expr_scope.at(iit->value);
+      const auto original_s = s;
       iit = iit->next;
       for (; iit != nullptr; iit = iit->next) {
         s = LCA(s, expr_scope.at(iit->value));
       }
+      if (s != original_s && node_to_expr.find(n) != node_to_expr.end()) {
+        // filter out exprs whose scope do not matter
+        Expr expr = node_to_expr[n];
+        if (!expr.as<OpNode>()) {
+          lifted_exprs.insert(expr);
+        }
+      }
+    }
+    if (n->new_scope) {
+      auto child_scope = std::make_shared<ScopeNode>(s);
+      expr_scope.insert({n, child_scope});
+    } else {
+      expr_scope.insert({n, s});
     }
-    expr_scope.insert({n, n->new_scope ? ChildScope(s) : s});
   }
   CHECK(global_scope_used);
-  return expr_scope;
+  return std::make_pair(expr_scope, lifted_exprs);
 }
 
-/* Special care is needed to handle local recursion.
- * Fill additionally take a (possibly null) Var argument,
- * If it is not null, Fill is required to bind the transformed result to that var.
- */
-class Fill : ExprFunctor<Expr(const Expr&, const Var&)> {
- public:
-  static Expr ToANormalForm(const Expr& e, const DependencyGraph& dg,
-                            std::unordered_map<DependencyGraph::Node*, Scope>* node_scope) {
-    Fill fi(dg, node_scope);
-    return fi.GetScope(e)->ll->Get(fi.VisitExpr(e));
-  }
-
- private:
-  const DependencyGraph& dg_;
-  std::unordered_map<DependencyGraph::Node*, Scope>* node_scope_;
-  std::unordered_map<Expr, Expr, ObjectPtrHash, ObjectPtrEqual> memo;
+Expr Fill::ToANormalForm(const Expr& e, const DependencyGraph& dg, NodeScopeMap* node_scope) {
+  Fill fi(dg, node_scope, nullptr);
+  return fi.GetScope(e)->ll->Get(fi.VisitExpr(e));
+}
 
-  Fill(const DependencyGraph& dg, std::unordered_map<DependencyGraph::Node*, Scope>* node_scope)
-      : dg_(dg), node_scope_(node_scope) {}
+// For basic block normal form, bind expressions only if the original expression's scope
+// should be lifted
+Expr Fill::ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                  NodeScopeMap* node_scope, ExprSet* lifted) {
+  Fill fi(dg, node_scope, lifted);
+  auto var = fi.VisitExpr(e);
+  return fi.GetScope(e)->ll->Get(var);
+}
 
-  Scope GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
+Scope Fill::GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
 
-  Scope GetSubScope(const Expr& e, size_t i) {
-    DependencyGraph::Node* n = dg_.expr_node.at(e);
-    auto h = n->children.head;
-    while (i != 0) {
-      CHECK(h);
-      --i;
-      h = h->next;
-    }
+Scope Fill::GetSubScope(const Expr& e, size_t i) {
+  DependencyGraph::Node* n = dg_.expr_node.at(e);
+  auto h = n->children.head;
+  while (i != 0) {
     CHECK(h);
-    return node_scope_->at(h->value);
+    --i;
+    h = h->next;
   }
+  CHECK(h);
+  return node_scope_->at(h->value);
+}
 
-  Expr VisitExpr(const Expr& e, const Var& v) final {
-    if (memo.count(e) == 0) {
-      memo.insert({e, ExprFunctor<Expr(const Expr&, const Var&)>::VisitExpr(e, v)});
-    } else if (v.defined()) {
-      GetScope(e)->ll->Push(v, memo.at(e));
-    }
-    auto ret = memo.at(e);
-    CHECK(IsAtomic(ret));
-    return ret;
+Expr Fill::VisitExpr(const Expr& e, const Var& v) {
+  if (memo.count(e) == 0) {
+    memo.insert({e, ExprFunctor<Expr(const Expr&, const Var&)>::VisitExpr(e, v)});
+  } else if (v.defined()) {
+    GetScope(e)->ll->Push(v, memo.at(e));
   }
+  auto ret = memo.at(e);
+  // if no include_set is specified, every expression should be atomic.
+  if (include_set_ == nullptr) CHECK(IsAtomic(ret));
+  return ret;
+}
 
-  Expr VisitExpr(const Expr& e) { return this->VisitExpr(e, Var()); }
+Expr Fill::VisitExpr(const Expr& e) { return this->VisitExpr(e, Var()); }
 
-  Expr Atomic(const Expr& e, const Var& v) { return v.defined() ? GetScope(e)->ll->Push(v, e) : e; }
+Expr Fill::Atomic(const Expr& e, const Var& v) {
+  return v.defined() ? GetScope(e)->ll->Push(v, e) : e;
+}
 
-  Expr Compound(const Expr& orig, const Expr& now, const Var& v) {
-    Var var = v.defined() ? v : Var(String("x"), Type());
+// Bind expression `now` to var `v` if the original expression is in the include set, or if
+// v is already defined (e.g. coming from a Let expression). Otherwise return `now` directly
+Expr Fill::Compound(const Expr& orig, const Expr& now, const Var& v) {
+  Var var = v.defined() ? v : Var(String("x"), Type());
+  bool not_included = include_set_ && include_set_->find(orig) == include_set_->end();
+  if (!v.defined() && not_included) {
+    return now;
+  } else {
     return GetScope(orig)->ll->Push(var, now);
   }
+}
 
-  Expr VisitExpr_(const CallNode* c, const Var& v) final {
-    Expr e = GetRef<Expr>(c);
-    std::vector<Expr> args;
-    for (const auto& a : c->args) {
-      args.push_back(VisitExpr(a));
-    }
-    return Compound(e, Call(VisitExpr(c->op), args, c->attrs, c->type_args), v);
+Expr Fill::VisitExpr_(const CallNode* c, const Var& v) {

Review comment:
       `final` is still used in the header at declaration https://en.cppreference.com/w/cpp/language/final#Syntax 

##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +189,89 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+using NodeScopeMap = std::unordered_map<DependencyGraph::Node*, Scope>;
+using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();
+  explicit ScopeNode(const Scope& parent) : level(1 + parent->level), parent(parent) {}
+  ScopeNode() : level(0) {}
+};
+
+/*! \brief Calculate the scope of nodes in the dependency graph by least common ancestor.
+ *
+ *  \param dg the input dependency graph
+ *  \param expr_scope the output node -> scope mapping for all nodes.
+ *  \param lifted_exprs the output set of expressions whose scope is lifted due to dependency
+ */
+std::pair<NodeScopeMap, ExprSet> CalcScope(const DependencyGraph& dg);
+
+/*! \brief find the least common ancestor of lhs scope and rhs scope.
+ */
+Scope LCA(Scope lhs, Scope rhs);
+
+/* Special care is needed to handle local recursion.
+ * Fill additionally take a (possibly null) Var argument,
+ * If it is not null, Fill is required to bind the transformed result to that var.
+ */
+class Fill : ExprFunctor<Expr(const Expr&, const Var&)> {
+ public:
+  static Expr ToANormalForm(const Expr& e, const DependencyGraph& dg, NodeScopeMap* node_scope);
+
+  // For basic block normal form, bind expressions only if the original expression's
+  // scope should be lifted
+  static Expr ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                     NodeScopeMap* node_scope, ExprSet* lifted);

Review comment:
       thanks for bringing this up. I was also thinking about how to best leverage the existing code without too much ad-hoc code or duplication. I think adding an inclusion/exclusion argument to `Fill` is still acceptable, and the Fill constructor is private anyway

##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+Expr ToBasicBlockNormalFormAux(const Expr& e) {

Review comment:
       I guess it's because of https://en.wikipedia.org/wiki/Basic_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] [incubator-tvm] MarisaKirisame commented on pull request #6152: [RELAY] Basic block normal form

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


   Thanks @eric-haibin-lin @mbrookhart @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.

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



[GitHub] [incubator-tvm] mbrookhart edited a comment on pull request #6152: [RELAY] Basic block normal form

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


   > I dont think it break single responsibility - the code is doing conversion and a single configuration param denote which conversion it is. Another way to think about it is that that visitor is declaring a 'scoped mutator' and toanf/tobbnf is two subclass of it.
   > We can refactor to the latter later if needed.
   
   That is exactly breaking the SRP, it's a class that does two mutually-exclusive conversions :) The SRP-compliant way to do this would be exactly what you said, a base class and two subclasses. That being said, since we're not allowing anyone to inherit from this class, and the constructor is private, it's probably not the end of the world, I'm cool with leaving it the way it is.
   


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

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



[GitHub] [incubator-tvm] mbrookhart commented on a change in pull request #6152: Basic block normal form

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



##########
File path: tests/python/relay/test_analysis_basic_block_normal_form.py
##########
@@ -0,0 +1,209 @@
+# 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 pytest
+import tvm
+from tvm import relay
+from tvm.relay.analysis import check_basic_block_normal_form
+
+def test_one_block():
+    x = relay.var('x')
+    y = relay.add(x, x)
+    z = relay.add(x, y)
+    assert check_basic_block_normal_form(z)
+
+def test_let():
+    x = relay.var('x')
+    y = relay.var('y')
+    body = relay.Let(y, x, y)
+    assert check_basic_block_normal_form(body)
+
+@pytest.mark.xfail(raises=tvm.error.TVMError)
+def test_invalid_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    """
+    The program below violates basic block normal form, as the scope of %shared
+    is ambiguous and should not be in that of true branch.
+
+    free_var %cond: bool
+    if (%cond) {
+      free_var %shared
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)
+    assert not check_basic_block_normal_form(body)
+
+def test_valid_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    shared_bound = relay.var('shared_bound', shape=(1,), dtype='float32')
+    body = relay.Let(shared, shared_bound, body)
+    """
+    The program below uses let binding to control the scope of %shared, which
+    follows the basic block normal form.
+
+    free_var %shared_bound: Tensor[(1), float32]
+    let %shared = %shared_bound;
+    free_var %cond: bool
+    if (%cond) {
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)
+    assert check_basic_block_normal_form(body)
+
+@pytest.mark.xfail(raises=tvm.error.TVMError)
+def test_invalid_if2():
+    """
+    fn (%x: float32) {
+      %0 = equal(%x, 2f);
+      if (%0) {
+        %1 = add(%x, 1f);
+        multiply(%1, 2f)
+      } else {
+        multiply(%1, 1f)
+      }
+    }
+    """
+    x = relay.var('x', shape=(), dtype='float32')
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    v1 = relay.add(x, one)
+    v2 = relay.equal(x, two)
+    true_branch = relay.multiply(v1, two)
+    false_branch = relay.multiply(v1, one)
+    body = relay.If(v2, true_branch, false_branch)
+    func = relay.Function([x], body)
+    assert not check_basic_block_normal_form(func)
+
+def test_valid_if2():
+    """
+    fn (%x: float32) {
+      let %v1 = add(%x, 1f);
+      %0 = equal(%x, 2f);
+      if (%0) {
+        multiply(%v1, 2f)
+      } else {
+        multiply(%v1, 1f)
+      }
+    }
+    """
+    x = relay.var('x', shape=(), dtype='float32')
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    v1 = relay.var('v1')
+    v2 = relay.equal(x, two)
+    true_branch = relay.multiply(v1, two)
+    false_branch = relay.multiply(v1, one)
+    body = relay.If(v2, true_branch, false_branch)
+    body = relay.Let(v1, relay.add(x, one), body)
+    func = relay.Function([x], body)
+    print(func)

Review comment:
       remove print

##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+Expr ToBasicBlockNormalFormAux(const Expr& e) {

Review comment:
       Why Basic Block normal form? Is there a more advanced version coming? Or should we just call it BlockNormalForm?

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)

Review comment:
       Remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)

Review comment:
       Remove print

##########
File path: tests/python/relay/test_analysis_basic_block_normal_form.py
##########
@@ -0,0 +1,209 @@
+# 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 pytest
+import tvm
+from tvm import relay
+from tvm.relay.analysis import check_basic_block_normal_form
+
+def test_one_block():
+    x = relay.var('x')
+    y = relay.add(x, x)
+    z = relay.add(x, y)
+    assert check_basic_block_normal_form(z)
+
+def test_let():
+    x = relay.var('x')
+    y = relay.var('y')
+    body = relay.Let(y, x, y)
+    assert check_basic_block_normal_form(body)
+
+@pytest.mark.xfail(raises=tvm.error.TVMError)
+def test_invalid_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    """
+    The program below violates basic block normal form, as the scope of %shared
+    is ambiguous and should not be in that of true branch.
+
+    free_var %cond: bool
+    if (%cond) {
+      free_var %shared
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)
+    assert not check_basic_block_normal_form(body)
+
+def test_valid_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    shared_bound = relay.var('shared_bound', shape=(1,), dtype='float32')
+    body = relay.Let(shared, shared_bound, body)
+    """
+    The program below uses let binding to control the scope of %shared, which
+    follows the basic block normal form.
+
+    free_var %shared_bound: Tensor[(1), float32]
+    let %shared = %shared_bound;
+    free_var %cond: bool
+    if (%cond) {
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)

Review comment:
       Remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)

Review comment:
       Remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)

Review comment:
       remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)

Review comment:
       Remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)

Review comment:
       Remove print

##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +189,89 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+using NodeScopeMap = std::unordered_map<DependencyGraph::Node*, Scope>;
+using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();

Review comment:
       Could we use a more expressive variable name, maybe `let_list`?

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)

Review comment:
       remove print

##########
File path: src/relay/transforms/to_a_normal_form.cc
##########
@@ -81,171 +70,187 @@ std::unordered_map<DependencyGraph::Node*, Scope> CalcScope(const DependencyGrap
       global_scope_used = true;
     } else {
       s = expr_scope.at(iit->value);
+      const auto original_s = s;
       iit = iit->next;
       for (; iit != nullptr; iit = iit->next) {
         s = LCA(s, expr_scope.at(iit->value));
       }
+      if (s != original_s && node_to_expr.find(n) != node_to_expr.end()) {
+        // filter out exprs whose scope do not matter
+        Expr expr = node_to_expr[n];
+        if (!expr.as<OpNode>()) {
+          lifted_exprs.insert(expr);
+        }
+      }
+    }
+    if (n->new_scope) {
+      auto child_scope = std::make_shared<ScopeNode>(s);
+      expr_scope.insert({n, child_scope});
+    } else {
+      expr_scope.insert({n, s});
     }
-    expr_scope.insert({n, n->new_scope ? ChildScope(s) : s});
   }
   CHECK(global_scope_used);
-  return expr_scope;
+  return std::make_pair(expr_scope, lifted_exprs);
 }
 
-/* Special care is needed to handle local recursion.
- * Fill additionally take a (possibly null) Var argument,
- * If it is not null, Fill is required to bind the transformed result to that var.
- */
-class Fill : ExprFunctor<Expr(const Expr&, const Var&)> {
- public:
-  static Expr ToANormalForm(const Expr& e, const DependencyGraph& dg,
-                            std::unordered_map<DependencyGraph::Node*, Scope>* node_scope) {
-    Fill fi(dg, node_scope);
-    return fi.GetScope(e)->ll->Get(fi.VisitExpr(e));
-  }
-
- private:
-  const DependencyGraph& dg_;
-  std::unordered_map<DependencyGraph::Node*, Scope>* node_scope_;
-  std::unordered_map<Expr, Expr, ObjectPtrHash, ObjectPtrEqual> memo;
+Expr Fill::ToANormalForm(const Expr& e, const DependencyGraph& dg, NodeScopeMap* node_scope) {
+  Fill fi(dg, node_scope, nullptr);
+  return fi.GetScope(e)->ll->Get(fi.VisitExpr(e));
+}
 
-  Fill(const DependencyGraph& dg, std::unordered_map<DependencyGraph::Node*, Scope>* node_scope)
-      : dg_(dg), node_scope_(node_scope) {}
+// For basic block normal form, bind expressions only if the original expression's scope
+// should be lifted
+Expr Fill::ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                  NodeScopeMap* node_scope, ExprSet* lifted) {
+  Fill fi(dg, node_scope, lifted);
+  auto var = fi.VisitExpr(e);
+  return fi.GetScope(e)->ll->Get(var);
+}
 
-  Scope GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
+Scope Fill::GetScope(const Expr& e) { return node_scope_->at(dg_.expr_node.at(e)); }
 
-  Scope GetSubScope(const Expr& e, size_t i) {
-    DependencyGraph::Node* n = dg_.expr_node.at(e);
-    auto h = n->children.head;
-    while (i != 0) {
-      CHECK(h);
-      --i;
-      h = h->next;
-    }
+Scope Fill::GetSubScope(const Expr& e, size_t i) {
+  DependencyGraph::Node* n = dg_.expr_node.at(e);
+  auto h = n->children.head;
+  while (i != 0) {
     CHECK(h);
-    return node_scope_->at(h->value);
+    --i;
+    h = h->next;
   }
+  CHECK(h);
+  return node_scope_->at(h->value);
+}
 
-  Expr VisitExpr(const Expr& e, const Var& v) final {
-    if (memo.count(e) == 0) {
-      memo.insert({e, ExprFunctor<Expr(const Expr&, const Var&)>::VisitExpr(e, v)});
-    } else if (v.defined()) {
-      GetScope(e)->ll->Push(v, memo.at(e));
-    }
-    auto ret = memo.at(e);
-    CHECK(IsAtomic(ret));
-    return ret;
+Expr Fill::VisitExpr(const Expr& e, const Var& v) {
+  if (memo.count(e) == 0) {
+    memo.insert({e, ExprFunctor<Expr(const Expr&, const Var&)>::VisitExpr(e, v)});
+  } else if (v.defined()) {
+    GetScope(e)->ll->Push(v, memo.at(e));
   }
+  auto ret = memo.at(e);
+  // if no include_set is specified, every expression should be atomic.
+  if (include_set_ == nullptr) CHECK(IsAtomic(ret));
+  return ret;
+}
 
-  Expr VisitExpr(const Expr& e) { return this->VisitExpr(e, Var()); }
+Expr Fill::VisitExpr(const Expr& e) { return this->VisitExpr(e, Var()); }
 
-  Expr Atomic(const Expr& e, const Var& v) { return v.defined() ? GetScope(e)->ll->Push(v, e) : e; }
+Expr Fill::Atomic(const Expr& e, const Var& v) {
+  return v.defined() ? GetScope(e)->ll->Push(v, e) : e;
+}
 
-  Expr Compound(const Expr& orig, const Expr& now, const Var& v) {
-    Var var = v.defined() ? v : Var(String("x"), Type());
+// Bind expression `now` to var `v` if the original expression is in the include set, or if
+// v is already defined (e.g. coming from a Let expression). Otherwise return `now` directly
+Expr Fill::Compound(const Expr& orig, const Expr& now, const Var& v) {
+  Var var = v.defined() ? v : Var(String("x"), Type());
+  bool not_included = include_set_ && include_set_->find(orig) == include_set_->end();
+  if (!v.defined() && not_included) {
+    return now;
+  } else {
     return GetScope(orig)->ll->Push(var, now);
   }
+}
 
-  Expr VisitExpr_(const CallNode* c, const Var& v) final {
-    Expr e = GetRef<Expr>(c);
-    std::vector<Expr> args;
-    for (const auto& a : c->args) {
-      args.push_back(VisitExpr(a));
-    }
-    return Compound(e, Call(VisitExpr(c->op), args, c->attrs, c->type_args), v);
+Expr Fill::VisitExpr_(const CallNode* c, const Var& v) {

Review comment:
       Why remove `final` from all of these functions?

##########
File path: tests/python/relay/test_analysis_basic_block_normal_form.py
##########
@@ -0,0 +1,209 @@
+# 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 pytest
+import tvm
+from tvm import relay
+from tvm.relay.analysis import check_basic_block_normal_form
+
+def test_one_block():
+    x = relay.var('x')
+    y = relay.add(x, x)
+    z = relay.add(x, y)
+    assert check_basic_block_normal_form(z)
+
+def test_let():
+    x = relay.var('x')
+    y = relay.var('y')
+    body = relay.Let(y, x, y)
+    assert check_basic_block_normal_form(body)
+
+@pytest.mark.xfail(raises=tvm.error.TVMError)
+def test_invalid_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    """
+    The program below violates basic block normal form, as the scope of %shared
+    is ambiguous and should not be in that of true branch.
+
+    free_var %cond: bool
+    if (%cond) {
+      free_var %shared
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)

Review comment:
       remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)

Review comment:
       Remove print

##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +189,89 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+using NodeScopeMap = std::unordered_map<DependencyGraph::Node*, Scope>;
+using ExprSet = std::unordered_set<Expr, ObjectPtrHash, ObjectPtrEqual>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();
+  explicit ScopeNode(const Scope& parent) : level(1 + parent->level), parent(parent) {}
+  ScopeNode() : level(0) {}
+};
+
+/*! \brief Calculate the scope of nodes in the dependency graph by least common ancestor.
+ *
+ *  \param dg the input dependency graph
+ *  \param expr_scope the output node -> scope mapping for all nodes.
+ *  \param lifted_exprs the output set of expressions whose scope is lifted due to dependency
+ */
+std::pair<NodeScopeMap, ExprSet> CalcScope(const DependencyGraph& dg);
+
+/*! \brief find the least common ancestor of lhs scope and rhs scope.
+ */
+Scope LCA(Scope lhs, Scope rhs);
+
+/* Special care is needed to handle local recursion.
+ * Fill additionally take a (possibly null) Var argument,
+ * If it is not null, Fill is required to bind the transformed result to that var.
+ */
+class Fill : ExprFunctor<Expr(const Expr&, const Var&)> {
+ public:
+  static Expr ToANormalForm(const Expr& e, const DependencyGraph& dg, NodeScopeMap* node_scope);
+
+  // For basic block normal form, bind expressions only if the original expression's
+  // scope should be lifted
+  static Expr ToBasicBlockNormalForm(const Expr& e, const DependencyGraph& dg,
+                                     NodeScopeMap* node_scope, ExprSet* lifted);

Review comment:
       I really like that you're reusing the infrastructure from A-Normal form to do this, there's a lot of overlap. I don't love that this class violates the Single Responsiblity Principle and that you have to introduce branches in a couple of the recursive functions to handle the two different cases. Given that this class has a private constructor, I'm not sure it 100% matters, but there might  be a cleaner solution?

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_gradient_if():
+    x = relay.var("a", shape=(1, 16))
+    y = relay.var("y", shape=(1, 16))
+    cond = relay.var("cond", shape=(), dtype='uint1')
+    net = relay.If(cond, x, x)
+    net = relay.add(x, net)
+    net = relay.Function([cond,x,y], net)
+    mod = tvm.IRModule.from_expr(net)
+    mod = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net=')
+    print(net)
+    print('mod=')
+    print(mod)
+    net_grad = relay.transform.gradient(mod["main"], mode='higher_order')
+    mod["main"] = net_grad
+    mod_grad = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net_grad=')
+    print(net_grad)
+    print('mod_grad=')
+    print(mod_grad)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_analysis_basic_block_normal_form.py
##########
@@ -0,0 +1,153 @@
+# 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 relay
+from tvm.relay.analysis import check_basic_block_normal_form
+
+def test_one_block():
+    x = relay.var('x')
+    y = relay.add(x, x)
+    z = relay.add(x, y)
+    assert check_basic_block_normal_form(z)
+
+def test_let():
+    x = relay.var('x')
+    y = relay.var('y')
+    body = relay.Let(y, x, y)
+    assert check_basic_block_normal_form(body)
+
+def test_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    """
+    The program below violates basic block normal form, as the scope of %shared
+    is ambiguous and should not be in that of true branch.
+
+    free_var %cond: bool
+    if (%cond) {
+      free_var %shared
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)
+    assert not check_basic_block_normal_form(body)

Review comment:
       :+1: Agreed.

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_gradient_if():
+    x = relay.var("a", shape=(1, 16))
+    y = relay.var("y", shape=(1, 16))
+    cond = relay.var("cond", shape=(), dtype='uint1')
+    net = relay.If(cond, x, x)
+    net = relay.add(x, net)
+    net = relay.Function([cond,x,y], net)
+    mod = tvm.IRModule.from_expr(net)
+    mod = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net=')
+    print(net)
+    print('mod=')
+    print(mod)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)

Review comment:
       Remove print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_gradient_if():
+    x = relay.var("a", shape=(1, 16))
+    y = relay.var("y", shape=(1, 16))
+    cond = relay.var("cond", shape=(), dtype='uint1')
+    net = relay.If(cond, x, x)
+    net = relay.add(x, net)
+    net = relay.Function([cond,x,y], net)
+    mod = tvm.IRModule.from_expr(net)
+    mod = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net=')
+    print(net)
+    print('mod=')
+    print(mod)
+    net_grad = relay.transform.gradient(mod["main"], mode='higher_order')
+    mod["main"] = net_grad
+    mod_grad = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net_grad=')
+    print(net_grad)
+    print('mod_grad=')
+    print(mod_grad)
+    assert check_basic_block_normal_form(mod_grad['main'])
+    assert check_basic_block_normal_form(mod['main'])
+
+def test_if():
+    def if_expr(x):
+        """
+        free_var %x: float32
+        %0 = equal(%x, 2f);
+        if (%0) {
+          %1 = add(%x, 1f);
+          multiply(%1, 2f)
+        } else {
+          multiply(%1, 1f)
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.add(x, one)
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        return body
+
+    def expected_if_expr(x):
+        """
+        free_var %x: float32
+        let %v1: float32 = add(%x, 1f /* ty=float32 */) /* ty=float32 */;
+        %0 = equal(%x, 2f /* ty=float32 */) /* ty=bool */;
+        if (%0) {
+          multiply(%v1, 2f /* ty=float32 */) /* ty=float32 */
+        } else {
+          multiply(%v1, 1f /* ty=float32 */) /* ty=float32 */
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.var('v1')
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        body = relay.Let(v1, relay.add(x, one), body)
+        return body
+
+    x = relay.var('x', shape=(), dtype='float32')
+    body = if_expr(x)
+    expected_body = expected_if_expr(x)
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_body, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+    func = relay.Function([x], body)
+    expected_func = relay.Function([x], expected_body)
+    bblock = run_opt_pass(func, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_func, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_higher_order_return():
+    x = relay.var('x', shape=(1,), dtype='float32')#, a)
+    y = relay.var('y', shape=(1,), dtype='float32')#, a)
+    z = relay.var('z', shape=(1,), dtype='float32')#, a)
+    x2 = relay.add(x, x)
+    func_a = relay.Function([y], relay.add(x2, y)) #, a, [a])
+    func_b = relay.Function([z], relay.add(x2, z)) #, a, [a])
+    body = relay.Tuple([func_a, func_b])
+    body = relay.Function([x], body)
+    """
+    fn (%x: Tensor[(1), float32]) {
+      %1 = fn (%y: Tensor[(1), float32]) {
+        %0 = add(%x, %x);
+        add(%0, %y)
+      };
+      %2 = fn (%z: Tensor[(1), float32]) {
+        add(%0, %z)
+      };
+      (%1, %2)
+    }
+    """
+
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+
+def test_higher_order_nested():
+    x = relay.var('x', dtype='float32', shape=(1,))
+    s = relay.var('s', dtype='float32', shape=(1,))
+    shared = relay.add(s, s)
+    func_true = relay.Function([x], relay.add(x, shared))
+    choice_t = relay.FuncType([], relay.scalar_type('bool'))
+    f = relay.Var('f', choice_t)
+    z = relay.Var('z')
+    body = relay.If(f(), func_true, relay.Function([z], relay.add(z, shared)))
+    top = relay.Function([f, s], body)
+    """
+    fn (%f: fn () -> bool, %s: Tensor[(1), float32]) {
+      %0 = %f();
+      if (%0) {
+        fn (%x: Tensor[(1), float32]) {
+          %1 = add(%s, %s);
+          add(%x, %1)
+        }
+      } else {
+        fn (%z) {
+          add(%z, %1)
+        }
+      }
+    }
+    """
+
+    bblock = run_opt_pass(top, transform.ToBasicBlockNormalForm())
+    print('top=')
+    print(top)
+    print('bblock=')
+    print(bblock)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_gradient_if():
+    x = relay.var("a", shape=(1, 16))
+    y = relay.var("y", shape=(1, 16))
+    cond = relay.var("cond", shape=(), dtype='uint1')
+    net = relay.If(cond, x, x)
+    net = relay.add(x, net)
+    net = relay.Function([cond,x,y], net)
+    mod = tvm.IRModule.from_expr(net)
+    mod = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net=')
+    print(net)
+    print('mod=')
+    print(mod)
+    net_grad = relay.transform.gradient(mod["main"], mode='higher_order')
+    mod["main"] = net_grad
+    mod_grad = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net_grad=')
+    print(net_grad)
+    print('mod_grad=')
+    print(mod_grad)
+    assert check_basic_block_normal_form(mod_grad['main'])
+    assert check_basic_block_normal_form(mod['main'])
+
+def test_if():
+    def if_expr(x):
+        """
+        free_var %x: float32
+        %0 = equal(%x, 2f);
+        if (%0) {
+          %1 = add(%x, 1f);
+          multiply(%1, 2f)
+        } else {
+          multiply(%1, 1f)
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.add(x, one)
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        return body
+
+    def expected_if_expr(x):
+        """
+        free_var %x: float32
+        let %v1: float32 = add(%x, 1f /* ty=float32 */) /* ty=float32 */;
+        %0 = equal(%x, 2f /* ty=float32 */) /* ty=bool */;
+        if (%0) {
+          multiply(%v1, 2f /* ty=float32 */) /* ty=float32 */
+        } else {
+          multiply(%v1, 1f /* ty=float32 */) /* ty=float32 */
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.var('v1')
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        body = relay.Let(v1, relay.add(x, one), body)
+        return body
+
+    x = relay.var('x', shape=(), dtype='float32')
+    body = if_expr(x)
+    expected_body = expected_if_expr(x)
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_body, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+    func = relay.Function([x], body)
+    expected_func = relay.Function([x], expected_body)
+    bblock = run_opt_pass(func, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_func, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_higher_order_return():
+    x = relay.var('x', shape=(1,), dtype='float32')#, a)
+    y = relay.var('y', shape=(1,), dtype='float32')#, a)
+    z = relay.var('z', shape=(1,), dtype='float32')#, a)
+    x2 = relay.add(x, x)
+    func_a = relay.Function([y], relay.add(x2, y)) #, a, [a])
+    func_b = relay.Function([z], relay.add(x2, z)) #, a, [a])
+    body = relay.Tuple([func_a, func_b])
+    body = relay.Function([x], body)
+    """
+    fn (%x: Tensor[(1), float32]) {
+      %1 = fn (%y: Tensor[(1), float32]) {
+        %0 = add(%x, %x);
+        add(%0, %y)
+      };
+      %2 = fn (%z: Tensor[(1), float32]) {
+        add(%0, %z)
+      };
+      (%1, %2)
+    }
+    """
+
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+
+def test_higher_order_nested():
+    x = relay.var('x', dtype='float32', shape=(1,))
+    s = relay.var('s', dtype='float32', shape=(1,))
+    shared = relay.add(s, s)
+    func_true = relay.Function([x], relay.add(x, shared))
+    choice_t = relay.FuncType([], relay.scalar_type('bool'))
+    f = relay.Var('f', choice_t)
+    z = relay.Var('z')
+    body = relay.If(f(), func_true, relay.Function([z], relay.add(z, shared)))
+    top = relay.Function([f, s], body)
+    """
+    fn (%f: fn () -> bool, %s: Tensor[(1), float32]) {
+      %0 = %f();
+      if (%0) {
+        fn (%x: Tensor[(1), float32]) {
+          %1 = add(%s, %s);
+          add(%x, %1)
+        }
+      } else {
+        fn (%z) {
+          add(%z, %1)
+        }
+      }
+    }
+    """
+
+    bblock = run_opt_pass(top, transform.ToBasicBlockNormalForm())
+    print('top=')
+    print(top)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)

Review comment:
       I'd love an expected graph on this test, but I don't think it's critical

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_gradient_if():
+    x = relay.var("a", shape=(1, 16))
+    y = relay.var("y", shape=(1, 16))
+    cond = relay.var("cond", shape=(), dtype='uint1')
+    net = relay.If(cond, x, x)
+    net = relay.add(x, net)
+    net = relay.Function([cond,x,y], net)
+    mod = tvm.IRModule.from_expr(net)
+    mod = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net=')
+    print(net)
+    print('mod=')
+    print(mod)
+    net_grad = relay.transform.gradient(mod["main"], mode='higher_order')
+    mod["main"] = net_grad
+    mod_grad = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net_grad=')
+    print(net_grad)
+    print('mod_grad=')
+    print(mod_grad)
+    assert check_basic_block_normal_form(mod_grad['main'])
+    assert check_basic_block_normal_form(mod['main'])
+
+def test_if():
+    def if_expr(x):
+        """
+        free_var %x: float32
+        %0 = equal(%x, 2f);
+        if (%0) {
+          %1 = add(%x, 1f);
+          multiply(%1, 2f)
+        } else {
+          multiply(%1, 1f)
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.add(x, one)
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        return body
+
+    def expected_if_expr(x):
+        """
+        free_var %x: float32
+        let %v1: float32 = add(%x, 1f /* ty=float32 */) /* ty=float32 */;
+        %0 = equal(%x, 2f /* ty=float32 */) /* ty=bool */;
+        if (%0) {
+          multiply(%v1, 2f /* ty=float32 */) /* ty=float32 */
+        } else {
+          multiply(%v1, 1f /* ty=float32 */) /* ty=float32 */
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.var('v1')
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        body = relay.Let(v1, relay.add(x, one), body)
+        return body
+
+    x = relay.var('x', shape=(), dtype='float32')
+    body = if_expr(x)
+    expected_body = expected_if_expr(x)
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_body, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+    func = relay.Function([x], body)
+    expected_func = relay.Function([x], expected_body)
+    bblock = run_opt_pass(func, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_func, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_higher_order_return():
+    x = relay.var('x', shape=(1,), dtype='float32')#, a)
+    y = relay.var('y', shape=(1,), dtype='float32')#, a)
+    z = relay.var('z', shape=(1,), dtype='float32')#, a)
+    x2 = relay.add(x, x)
+    func_a = relay.Function([y], relay.add(x2, y)) #, a, [a])
+    func_b = relay.Function([z], relay.add(x2, z)) #, a, [a])
+    body = relay.Tuple([func_a, func_b])
+    body = relay.Function([x], body)
+    """
+    fn (%x: Tensor[(1), float32]) {
+      %1 = fn (%y: Tensor[(1), float32]) {
+        %0 = add(%x, %x);
+        add(%0, %y)
+      };
+      %2 = fn (%z: Tensor[(1), float32]) {
+        add(%0, %z)
+      };
+      (%1, %2)
+    }
+    """
+
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)

Review comment:
       Remove Print

##########
File path: tests/python/relay/test_pass_to_basic_block_normal_form.py
##########
@@ -0,0 +1,544 @@
+# 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 pytest
+import numpy as np
+import tvm
+from tvm import te
+from tvm import relay
+from tvm.relay.analysis import detect_feature
+from tvm.relay import op, create_executor, transform
+from tvm.relay.prelude import Prelude
+from tvm.relay.testing import add_nat_definitions, count
+from tvm.relay.analysis import Feature
+from tvm.relay.analysis import check_basic_block_normal_form
+
+
+def run_opt_pass(expr, passes):
+    passes = passes if isinstance(passes, list) else [passes]
+    mod = tvm.IRModule.from_expr(expr)
+    seq = tvm.transform.Sequential(passes)
+    with tvm.transform.PassContext(opt_level=3):
+       mod = seq(mod)
+    entry = mod["main"]
+    return entry if isinstance(expr, relay.Function) else entry.body
+
+
+def check_eval(expr, expected_result, mod=None, rtol=1e-07):
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+
+    result = intrp.evaluate(expr)
+    np.testing.assert_allclose(result.asnumpy(), expected_result, rtol=rtol)
+
+
+def test_no_explicit_bind():
+    x = relay.const(1)
+    y = op.add(x, x)
+    z = op.add(y, y)
+    f = relay.Function([], op.add(z, z))
+    print(f)
+    """
+    fn () {
+      %0 = add(1, 1);
+      %1 = add(%0, %0);
+      add(%1, %1)
+    }
+    """
+    assert not Feature.fLet in detect_feature(f)
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    print(bblock)
+    assert Feature.fLet not in detect_feature(bblock)
+    check_eval(f(), 8.0)
+    check_eval(bblock(), 8.0)
+    assert check_basic_block_normal_form(bblock)
+
+def test_top_level_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    z = relay.var('z', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    z2 = relay.add(z, z)
+    true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+    false_branch = relay.If(cond_f, z2, one)
+    body = relay.If(x, true_branch, false_branch)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %z: float32
+        %0 = add(%z, %z);
+        free_var %y: float32
+        %1 = add(%y, %y);
+        add(%0, %1)
+      } else {
+        add(3f, %1)
+      }
+    } else {
+      if (False) {
+        %0
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        z = relay.var('z', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        z2 = relay.var('z2')
+        true_branch = relay.If(cond_t, relay.add(z2, y2), relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, z2, one)
+        body = relay.If(x, true_branch, false_branch)
+        body = relay.Let(z2, relay.add(z, z), body)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    """
+    free_var %z: float32
+    let %x: float32 = add(%z, %z) /* ty=float32 */;
+    free_var %x1: bool
+    if (%x1) {
+      free_var %y: float32
+      let %x2: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        add(%x, %x2) /* ty=float32 */
+      } else {
+        add(3f /* ty=float32 */, %x2) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        %x
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+
+def test_nested_if():
+    x = relay.var('x', shape=(), dtype='bool')
+    y = relay.var('y', shape=(), dtype='float32')
+    cond_t = relay.const(True)
+    cond_f = relay.const(False)
+    one = relay.const(1, dtype='float32')
+    two = relay.const(2, dtype='float32')
+    three = relay.const(3, dtype='float32')
+    y2 = relay.add(y, y)
+    true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+    false_branch = relay.If(cond_f, two, one)
+    body = relay.If(x, true_branch, false_branch)
+    print(body)
+    """
+    free_var %x: bool
+    if (%x) {
+      if (True) {
+        free_var %y: float32
+        %0 = add(%y, %y);
+        %0
+      } else {
+        add(3f, %0)
+      }
+    } else {
+      if (False) {
+        2f
+      } else {
+        1f
+      }
+    }
+    """
+    def expected():
+        x = relay.var('x', shape=(), dtype='bool')
+        y = relay.var('y', shape=(), dtype='float32')
+        cond_t = relay.const(True)
+        cond_f = relay.const(False)
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        three = relay.const(3, dtype='float32')
+        y2 = relay.var('y2')
+        true_branch = relay.If(cond_t, y2, relay.add(three, y2))
+        true_branch = relay.Let(y2, relay.add(y, y), true_branch)
+        false_branch = relay.If(cond_f, two, one)
+        body = relay.If(x, true_branch, false_branch)
+        return body
+
+    bblock = run_opt_pass(body, [transform.ToBasicBlockNormalForm()])
+    print(bblock)
+    """
+    free_var %x: bool
+    if (%x) {
+      free_var %y: float32
+      let %x1: float32 = add(%y, %y) /* ty=float32 */;
+      if (True /* ty=bool */) {
+        %x1
+      } else {
+        add(3f /* ty=float32 */, %x1) /* ty=float32 */
+      }
+    } else {
+      if (False /* ty=bool */) {
+        2f /* ty=float32 */
+      } else {
+        1f /* ty=float32 */
+      }
+    }
+    """
+    expected_output = run_opt_pass(expected(), transform.InferType())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    print('expected_output=')
+    print(expected_output)
+    assert tvm.ir.structural_equal(bblock, expected_output, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+
+# make sure we do not infinite loop.
+# it is too large so we won't check for the exact program.
+def test_recursion():
+    """
+    Program:
+       let f(n: i32) -> i32 = {
+          m = (n * 2)
+          if (n == 0) {
+              return m;
+          } else {
+              return m + f(n - 1);
+          }
+       }
+       f(5);
+    """
+    mod = tvm.IRModule()
+    i64 = relay.TensorType((), 'int64')
+    f = relay.GlobalVar("f")
+    n = relay.Var("n", i64)
+    m = n * relay.const(2, 'int64')
+    cond = relay.equal(n, relay.const(0, 'int64'))
+    false_branch = m + f(n - relay.const(1, 'int64'))
+    funcbody = relay.If(cond, m, false_branch)
+    value = relay.Function([n], funcbody, i64, [])
+    mod[f] = value
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    old_f = mod[f]
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    f = mod[f]
+    print('old_f=')
+    print(old_f)
+    print('f=')
+    print(f)
+    check_eval(f(relay.const(5, 'int64')), 30.0, mod=mod)
+    assert check_basic_block_normal_form(f)
+
+def test_ref():
+    i = relay.Var('i')
+    iv = relay.Var('iv')
+    u = relay.Var('u')
+    uv = relay.Var('uv')
+    body = relay.add(iv, uv)
+    body = relay.Let(uv, relay.RefRead(i), body)
+    body = relay.Let(u, relay.RefWrite(i, relay.const(2)), body)
+    body = relay.Let(iv, relay.RefRead(i), body)
+    body = relay.Let(i, relay.RefCreate(relay.const(1)), body)
+    check_eval(body, 3)
+    opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('opt_body=')
+    print(opt_body)
+    check_eval(opt_body, 3)
+    assert check_basic_block_normal_form(opt_body)
+
+
+def test_nat_add():
+    mod = tvm.IRModule()
+    p = Prelude(mod)
+    add_nat_definitions(p)
+    nat = p.nat
+    add = p.add
+    s = p.s
+    z = p.z
+    ctx = tvm.context("llvm", 0)
+    intrp = create_executor(mod=mod, ctx=ctx, target="llvm")
+    assert mod[add].checked_type == relay.FuncType([nat(), nat()], nat())
+    assert count(p, intrp.evaluate(add(s(z()), s(z())))) == 2
+    expr = add(s(z()), s(z()))
+    f = relay.GlobalVar("f")
+    mod[f] = relay.Function([], expr)
+    mod = transform.ToBasicBlockNormalForm()(mod)
+    opt_expr = mod["f"]
+    print('expr=', expr)
+    print('opt_expr=', opt_expr)
+    assert count(p, intrp.evaluate(opt_expr.body)) == 2
+    assert not Feature.fLet in detect_feature(mod[add])
+    assert check_basic_block_normal_form(opt_expr)
+
+def test_let():
+    def test_let1():
+        x = relay.Var("x")
+        c = relay.const(4.0, 'float32')
+        body = relay.Let(x, c, x)
+        body = run_opt_pass(body, transform.InferType())
+        """
+        let %x: float32 = 4f /* ty=float32 */;
+        %x
+        """
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+        
+    def test_let1_1():
+        x = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(x, d, relay.add(x,x))
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+    
+    def test_let2():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(y, x, x)
+        body = relay.Let(x, d, body)
+        body = run_opt_pass(body, transform.InferType())
+        check_eval(body, 4)
+
+        def expected():
+            x = relay.Var("x")
+            y = relay.Var("y")
+            d = relay.const(4.0, 'float32')
+            body = relay.Let(y, x, y)
+            body = relay.Let(x, d, body)
+            return body
+
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        expected_body = run_opt_pass(expected(), transform.InferType())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(opt_body, expected_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    def test_let3():
+        x = relay.Var("x")
+        y = relay.Var("y")
+        z = relay.Var("z")
+        c = relay.const(3.0, 'float32')
+        d = relay.const(4.0, 'float32')
+        body = relay.Let(z, x + y, x + z)
+        body = relay.Let(x, d, body)
+        body = relay.Let(y, c, body)
+        body = run_opt_pass(body, transform.InferType())
+        opt_body = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+        print('body=')
+        print(body)
+        print('opt_body=')
+        print(opt_body)
+        assert tvm.ir.structural_equal(body, opt_body)
+        assert check_basic_block_normal_form(opt_body)
+
+    test_let1()
+    test_let1_1()
+    test_let2()
+    test_let3()
+
+def test_function():
+    t = relay.TensorType((), 'float32')
+    x = relay.Var("x", t)
+    f = relay.Function([x], x + x)
+    d = relay.const(4.0, 'float32')
+    bblock = run_opt_pass(f, transform.ToBasicBlockNormalForm())
+    assert isinstance(bblock, relay.Function)
+    check_eval(f(d), 8)
+    check_eval(bblock(d), 8)
+    print('f=')
+    print(f)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_gradient_if():
+    x = relay.var("a", shape=(1, 16))
+    y = relay.var("y", shape=(1, 16))
+    cond = relay.var("cond", shape=(), dtype='uint1')
+    net = relay.If(cond, x, x)
+    net = relay.add(x, net)
+    net = relay.Function([cond,x,y], net)
+    mod = tvm.IRModule.from_expr(net)
+    mod = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net=')
+    print(net)
+    print('mod=')
+    print(mod)
+    net_grad = relay.transform.gradient(mod["main"], mode='higher_order')
+    mod["main"] = net_grad
+    mod_grad = relay.transform.ToBasicBlockNormalForm()(mod)
+    print('net_grad=')
+    print(net_grad)
+    print('mod_grad=')
+    print(mod_grad)
+    assert check_basic_block_normal_form(mod_grad['main'])
+    assert check_basic_block_normal_form(mod['main'])
+
+def test_if():
+    def if_expr(x):
+        """
+        free_var %x: float32
+        %0 = equal(%x, 2f);
+        if (%0) {
+          %1 = add(%x, 1f);
+          multiply(%1, 2f)
+        } else {
+          multiply(%1, 1f)
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.add(x, one)
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        return body
+
+    def expected_if_expr(x):
+        """
+        free_var %x: float32
+        let %v1: float32 = add(%x, 1f /* ty=float32 */) /* ty=float32 */;
+        %0 = equal(%x, 2f /* ty=float32 */) /* ty=bool */;
+        if (%0) {
+          multiply(%v1, 2f /* ty=float32 */) /* ty=float32 */
+        } else {
+          multiply(%v1, 1f /* ty=float32 */) /* ty=float32 */
+        }
+        """
+        one = relay.const(1, dtype='float32')
+        two = relay.const(2, dtype='float32')
+        v1 = relay.var('v1')
+        v2 = relay.equal(x, two)
+        true_branch = relay.multiply(v1, two)
+        false_branch = relay.multiply(v1, one)
+        body = relay.If(v2, true_branch, false_branch)
+        body = relay.Let(v1, relay.add(x, one), body)
+        return body
+
+    x = relay.var('x', shape=(), dtype='float32')
+    body = if_expr(x)
+    expected_body = expected_if_expr(x)
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_body, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock, map_free_vars=True)
+    assert check_basic_block_normal_form(bblock)
+
+    func = relay.Function([x], body)
+    expected_func = relay.Function([x], expected_body)
+    bblock = run_opt_pass(func, transform.ToBasicBlockNormalForm())
+    expected_bblock = run_opt_pass(expected_func, transform.InferType())
+    assert tvm.ir.structural_equal(bblock, expected_bblock)
+    assert check_basic_block_normal_form(bblock)
+
+def test_higher_order_return():
+    x = relay.var('x', shape=(1,), dtype='float32')#, a)
+    y = relay.var('y', shape=(1,), dtype='float32')#, a)
+    z = relay.var('z', shape=(1,), dtype='float32')#, a)
+    x2 = relay.add(x, x)
+    func_a = relay.Function([y], relay.add(x2, y)) #, a, [a])
+    func_b = relay.Function([z], relay.add(x2, z)) #, a, [a])
+    body = relay.Tuple([func_a, func_b])
+    body = relay.Function([x], body)
+    """
+    fn (%x: Tensor[(1), float32]) {
+      %1 = fn (%y: Tensor[(1), float32]) {
+        %0 = add(%x, %x);
+        add(%0, %y)
+      };
+      %2 = fn (%z: Tensor[(1), float32]) {
+        add(%0, %z)
+      };
+      (%1, %2)
+    }
+    """
+
+    bblock = run_opt_pass(body, transform.ToBasicBlockNormalForm())
+    print('body=')
+    print(body)
+    print('bblock=')
+    print(bblock)
+    assert check_basic_block_normal_form(bblock)

Review comment:
       I'd love an expected graph on this test, but I don't think it's critical




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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #6152: [WIP] Basic block normal form

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



##########
File path: src/relay/transforms/to_a_normal_form.cc
##########
@@ -50,10 +50,17 @@ Scope LCA(Scope lhs, Scope rhs) {
   return lhs;
 }
 
-std::unordered_map<DependencyGraph::Node*, Scope> CalcScope(const DependencyGraph& dg) {
-  std::unordered_map<DependencyGraph::Node*, Scope> expr_scope;
+// return a set of Exprs whose scope should be lifted to due dependencies.
+void CalcScope(const DependencyGraph& dg,
+               std::unordered_map<DependencyGraph::Node*, Scope>* expr_scope,

Review comment:
       Why this change?




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

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



[GitHub] [incubator-tvm] tqchen commented on pull request #6152: [WIP] Basic block normal form

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


   Thanks @eric-haibin-lin , it would also be useful to have a verification analysis util that verifies the IR is in the basic block form


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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #6152: Basic block normal form

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



##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +188,37 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();
+  explicit ScopeNode(const Scope& parent) : level(1 + parent->level), parent(parent) {}
+  ScopeNode() : level(0) {}
+};
+
+/*! \brief Calculate the scope of nodes in the dependency graph by least common ancestor.
+ *
+ *  \param dg the input dependency graph
+ *  \param expr_scope the output node -> scope mapping for all nodes.
+ *  \param lifted_exprs the output set of expressions whose scope is lifted due to dependency
+ */
+void CalcScope(const DependencyGraph& dg,
+               std::unordered_map<DependencyGraph::Node*, Scope>* expr_scope,

Review comment:
       I think we should disable cppint and use tuple, or just do a few typedef to shorten stuff.




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

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



[GitHub] [incubator-tvm] jroesch commented on pull request #6152: [WIP] Basic block normal form

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


   cc @MarisaKirisame 


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

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



[GitHub] [incubator-tvm] ANSHUMAN87 commented on pull request #6152: [WIP] Basic block normal form

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


   @eric-haibin-lin : Thanks for the PR! I believe this feature will be really helpful for certain detection & mutation process.
   Please let me know, once your PR is ready for review. Would like to get a hold on 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.

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



[GitHub] [incubator-tvm] mbrookhart commented on pull request #6152: [RELAY] Basic block normal form

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


   > I dont think it break single responsibility - the code is doing conversion and a single configuration param denote which conversion it is. Another way to think about it is that that visitor is declaring a 'scoped mutator' and toanf/tobbnf is two subclass of it.
   > We can refactor to the latter later if needed.
   
   That is exactly breaking the SRP :) The SRP-compliant way to do this would be exactly what you said, a base class and two subclasses. That being said, since we're not allowing anyone to inherit from this class, and the constructor is private, it's probably not the end of the world, I'm cool with leaving it the way it is.
   


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

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



[GitHub] [incubator-tvm] eric-haibin-lin commented on a change in pull request #6152: Basic block normal form

Posted by GitBox <gi...@apache.org>.
eric-haibin-lin commented on a change in pull request #6152:
URL: https://github.com/apache/incubator-tvm/pull/6152#discussion_r463983464



##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +188,37 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();
+  explicit ScopeNode(const Scope& parent) : level(1 + parent->level), parent(parent) {}
+  ScopeNode() : level(0) {}
+};
+
+/*! \brief Calculate the scope of nodes in the dependency graph by least common ancestor.
+ *
+ *  \param dg the input dependency graph
+ *  \param expr_scope the output node -> scope mapping for all nodes.
+ *  \param lifted_exprs the output set of expressions whose scope is lifted due to dependency
+ */
+void CalcScope(const DependencyGraph& dg,
+               std::unordered_map<DependencyGraph::Node*, Scope>* expr_scope,

Review comment:
       @MarisaKirisame thanks for the review! For BBlock I want to get a set of expressions whose scope should be lifted and rewritten with let bindings. I found `CalcScope` already performs finding the LCA for all nodes in the dependency graph, so i just added a few lines of code so that the set of corresponding expressions are also returned by this function. However, setting the return type to `std::pair<std::unordered_map<node, scope> std::unordered_set<expr>>` causes cpplint to fails due to the long length. So I instead used pointers for both outputs. Please let me know the preferred way for returning both outputs. Thanks! 




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

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



[GitHub] [incubator-tvm] junrushao1994 removed a comment on pull request #6152: [WIP] Basic block normal form

Posted by GitBox <gi...@apache.org>.
junrushao1994 removed a comment on pull request #6152:
URL: https://github.com/apache/incubator-tvm/pull/6152#issuecomment-665942064


   Ping me if the PR is ready to review :-)


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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #6152: Basic block normal form

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



##########
File path: tests/python/relay/test_analysis_basic_block_normal_form.py
##########
@@ -0,0 +1,153 @@
+# 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 relay
+from tvm.relay.analysis import check_basic_block_normal_form
+
+def test_one_block():
+    x = relay.var('x')
+    y = relay.add(x, x)
+    z = relay.add(x, y)
+    assert check_basic_block_normal_form(z)
+
+def test_let():
+    x = relay.var('x')
+    y = relay.var('y')
+    body = relay.Let(y, x, y)
+    assert check_basic_block_normal_form(body)
+
+def test_if():
+    cond = relay.var('cond', dtype='bool', shape=())
+    shared = relay.var('shared')
+    true_branch = shared
+    false_branch = relay.add(shared, shared)
+    body = relay.If(cond, true_branch, false_branch)
+    """
+    The program below violates basic block normal form, as the scope of %shared
+    is ambiguous and should not be in that of true branch.
+
+    free_var %cond: bool
+    if (%cond) {
+      free_var %shared
+      %shared
+    } else {
+      add(%shared, %shared)
+    }
+    """
+    print(body)
+    assert not check_basic_block_normal_form(body)

Review comment:
       is_basic_block_normal_form test if something is or is not in bbnf.
   check_basic_block_normal_form should assert it, and perhaps give more useful error message

##########
File path: src/relay/transforms/to_basic_block_normal_form.cc
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.
+ */
+
+/*!
+ *
+ * \file to_basic_block_normal_form.cc
+ *
+ * \brief Turn an expression to the basic normal form.
+ */
+#include <tvm/relay/analysis.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/transform.h>
+#include <tvm/support/logging.h>
+
+#include "../../support/arena.h"
+#include "../analysis/dependency_graph.h"
+#include "let_list.h"
+#include "pass_util.h"
+
+namespace tvm {
+namespace relay {
+
+/* Fill expressions based on each scope's let list. Different from FillANF,
+ * only expressions with lifted scope will be pushed to the let list.
+ */
+class FillBasicBlock : ExprFunctor<Expr(const Expr&, const Var&)> {

Review comment:
       can you reuse the Fill in to_anf? the code is almost identically the same, and duplicating it will make maintenance significantly harder.




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

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



[GitHub] [incubator-tvm] eric-haibin-lin commented on a change in pull request #6152: Basic block normal form

Posted by GitBox <gi...@apache.org>.
eric-haibin-lin commented on a change in pull request #6152:
URL: https://github.com/apache/incubator-tvm/pull/6152#discussion_r464011977



##########
File path: src/relay/transforms/pass_util.h
##########
@@ -184,6 +188,37 @@ struct TreeBranchNode : TreeNode<ConditionObjectPtr> {
   ~TreeBranchNode() {}
 };
 
+struct ScopeNode;
+using Scope = std::shared_ptr<ScopeNode>;
+
+/* Invariant: when parent is null level is 0
+ * Invariant: when parent is not null level is 1 + parent->level
+ */
+struct ScopeNode {
+  // the level of the scope
+  size_t level;
+  // the parent scope
+  Scope parent;
+  // the corresponding let list which holds all let bindings in the scope
+  std::shared_ptr<LetList> ll = std::make_shared<LetList>();
+  explicit ScopeNode(const Scope& parent) : level(1 + parent->level), parent(parent) {}
+  ScopeNode() : level(0) {}
+};
+
+/*! \brief Calculate the scope of nodes in the dependency graph by least common ancestor.
+ *
+ *  \param dg the input dependency graph
+ *  \param expr_scope the output node -> scope mapping for all nodes.
+ *  \param lifted_exprs the output set of expressions whose scope is lifted due to dependency
+ */
+void CalcScope(const DependencyGraph& dg,
+               std::unordered_map<DependencyGraph::Node*, Scope>* expr_scope,

Review comment:
       Ok I'll update the code
   




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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on pull request #6152: [RELAY] Basic block normal form

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


   I dont think it break single responsibility - the code is doing conversion and a single configuration param denote which conversion it is. Another way to think about it is that that visitor is declaring a 'scoped mutator' and toanf/tobbnf is two subclass of it.
   We can refactor to the latter later if needed.


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

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



[GitHub] [incubator-tvm] MarisaKirisame commented on pull request #6152: [RELAY] Basic block normal form

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


   @eric-haibin-lin please at me when it is done. I will merge it then.


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

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



[GitHub] [incubator-tvm] tqchen edited a comment on pull request #6152: [WIP] Basic block normal form

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


   cc @MarisaKirisame @mbrookhart @electriclilies @jroesch @junrushao1994  would be great if you can help to take a look  at the PR


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

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