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/03/27 02:32:43 UTC

[GitHub] [incubator-tvm] tqchen opened a new pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

tqchen opened a new pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154
 
 
   This PR introduces a new way to handle structural equality
   for both TIR and relay nodes in an extensive way.
   
   - Each object can now register an optional SEqualReduce function, which
     describes how to reduce its structural equality to another instance
     into equality of the children.
   - Optionally, the object can choose to allow remapping of vars(e.g. function parameters)
     by calling DefEqual
   - We implemented a non-recursive structural equality checker that
     recursively traverses the objects and does the structural equality checking.
   
   This PR also fixes a few potential problems in previous relay's AlphaEqual.
   
   - In particular, the new structural equality relation will be communicative.
   - It is can be dangerous to use same_as relation to quickly check equality,
     demonstrated by the following case. (%x, %y) are shared vars between two functions.
   
   - function0: fn (%x, %y) { %x + %y }
   - function1. fn (%y, %x) { %x + %y }
   
   The new structural equal is intented to supersede AlphaEqual and AttrsEqual.
   
   Follow-up PRs should be performed to redirect the existing usages, and removes
   the corresponding implementation.
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399448385
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   I see, make sense to me.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399839877
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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 src/node/structural_equal.cc
+ */
+#include <tvm/node/structural_equal.h>
+#include <tvm/node/reflection.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/node.h>
+#include <tvm/runtime/registry.h>
+
+#include <unordered_map>
+
+namespace tvm {
+
+// Define the dispatch functio here since primary user is in this file.
+bool ReflectionVTable::
+SEqualReduce(const Object* self, const Object* other, SEqualReducer equal) const {
+  uint32_t tindex = self->type_index();
+  if (tindex >= fsequal_.size() || fsequal_[tindex] == nullptr) {
+    LOG(FATAL) << "TypeError: SEqualReduce of " << self->GetTypeKey()
+        << " is not registered via TVM_REGISTER_NODE_TYPE";
+  }
+  return fsequal_[tindex](self, other, equal);
+}
+
+/*!
+ * \brief A non recursive stack based SEqual handler that can remaps vars.
+ *
+ *  This handler pushs the Object equality cases into a stack, and
+ *  traverses the stack to expand the necessary children that need to be checked.
+ *
+ *  The order of SEqual being called is the same as the order as if we
+ *  eagerly do recursive calls in SEqualReduce.
+ */
+class RemapVarSEqualHandler :
+      public SEqualReducer::Handler {
+ public:
+  explicit RemapVarSEqualHandler(bool assert_mode)
+      : assert_mode_(assert_mode) {}
+
+  bool SEqualReduce(const ObjectRef& lhs, const ObjectRef& rhs, bool map_free_vars) final {
+    // We cannot use check lhs.same_as(rhs) to check equality.
+    // if we choose to enable var remapping.
+    //
+    // Counter example below (%x, %y) are shared vars
+    // between the two functions(possibly before/after rewriting).
+    //
+    // - function0: fn (%x, %y) { %x + %y }
+    // - function1. fn (%y, %x) { %x + %y }
+    //
+    // Because we choose to enable var remapping,
+    // %x is mapped to %y, and %y is mapped to %x,
+    // the body of the function no longer means the same thing.
+    //
+    // Take away: We can either choose only compare Var by address,
+    // in which case we can use same_as for quick checking,
+    // or we have to run deep comparison and avoid to use same_as checks.
+    auto run = [=]() {
+      if (!lhs.defined() && !rhs.defined()) return true;
 
 Review comment:
   @tqchen I am not sure, but I feel like there might be a bug here when both `lhs` and `rhs` are not defined, because we wont execute Line86. However Line 111 expects L86 is executed. Am I right?
   
   If this is true, we may want to move Line77 before Line106, or we do `emplace_back(ObjectRef(nullptr), ...)` before `return true` and `return true` when both `lhs` and `rhs` are not defined at Line178.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399481397
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   note that zx are short hand for `(vx+vx)` here (need to read the entire function), they are not separate variable. So the comparison  here is a comparing a diamond dag of (x+x) * (x+x) vs a tree. In the case of TIR(where graph node semantics is not yet applied) both are simply treated as a tree atm

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399418044
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -189,16 +201,19 @@ class RemapVarSEqualHandler :
     bool map_free_vars;
     /*! \brief Whether the children has been expanded via SEqualReduce */
     bool children_expanded{false};
+    /*! \brief whether the task is about graph equality(need remap). */
+    bool graph_equal{false};
 
     Task() = default;
     Task(ObjectRef lhs, ObjectRef rhs, bool map_free_vars)
         : lhs(lhs), rhs(rhs), map_free_vars(map_free_vars) {}
   };
-
   // list of pending tasks to be pushed to the stack.
   std::vector<Task> pending_tasks_;
   // Internal task stack to executed the task
   std::vector<Task> task_stack_;
+  // record current stack top
 
 Review comment:
   should we also use /*! \brief */ comment style here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399422483
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -189,16 +201,19 @@ class RemapVarSEqualHandler :
     bool map_free_vars;
     /*! \brief Whether the children has been expanded via SEqualReduce */
     bool children_expanded{false};
+    /*! \brief whether the task is about graph equality(need remap). */
+    bool graph_equal{false};
 
     Task() = default;
     Task(ObjectRef lhs, ObjectRef rhs, bool map_free_vars)
         : lhs(lhs), rhs(rhs), map_free_vars(map_free_vars) {}
   };
-
   // list of pending tasks to be pushed to the stack.
   std::vector<Task> pending_tasks_;
   // Internal task stack to executed the task
   std::vector<Task> task_stack_;
+  // record current stack top
 
 Review comment:
   Given that this is private and won't generate docs, perhaps we can just go with this way

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] junrushao1994 commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
junrushao1994 commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399550850
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -0,0 +1,103 @@
+# 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 te
+
+
+def test_exprs():
+    # save load json
+    x = tvm.tir.const(1, "int32")
+    y = tvm.tir.const(10, "int32")
+    vx = te.var("x")
+    vy = te.var("y")
+    vz = te.var("z")
+    # test assert trigger.
+    try:
+        tvm.ir.assert_structural_equal(x, y)
 
 Review comment:
   use [`pytest.raises`](https://docs.pytest.org/en/latest/assert.html)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399434631
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   Note that the main point of difference is whether we treat a node as graph node or normal one. the current semantics on the TIR is to treat expr nodes as normal nodes. We could change the definition by marking all TIR expr as graph nodes as well, which will give the other behavior.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604806944
 
 
   One trick I had done for alphaequal is, there is an addition flag - must_equal_true, which is default to false. if the flag is on, and alphaequal return false, then it will report a small unequal pair of program fragment.
   This make writing unit test easier, as, when somethinig is wrong, we can pinpoint a small wrong fragment.
   Should we have the same thing for structural equal?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399854584
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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 src/node/structural_equal.cc
+ */
+#include <tvm/node/structural_equal.h>
+#include <tvm/node/reflection.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/node.h>
+#include <tvm/runtime/registry.h>
+
+#include <unordered_map>
+
+namespace tvm {
+
+// Define the dispatch functio here since primary user is in this file.
+bool ReflectionVTable::
+SEqualReduce(const Object* self, const Object* other, SEqualReducer equal) const {
+  uint32_t tindex = self->type_index();
+  if (tindex >= fsequal_.size() || fsequal_[tindex] == nullptr) {
+    LOG(FATAL) << "TypeError: SEqualReduce of " << self->GetTypeKey()
+        << " is not registered via TVM_REGISTER_NODE_TYPE";
+  }
+  return fsequal_[tindex](self, other, equal);
+}
+
+/*!
+ * \brief A non recursive stack based SEqual handler that can remaps vars.
+ *
+ *  This handler pushs the Object equality cases into a stack, and
+ *  traverses the stack to expand the necessary children that need to be checked.
+ *
+ *  The order of SEqual being called is the same as the order as if we
+ *  eagerly do recursive calls in SEqualReduce.
+ */
+class RemapVarSEqualHandler :
+      public SEqualReducer::Handler {
+ public:
+  explicit RemapVarSEqualHandler(bool assert_mode)
+      : assert_mode_(assert_mode) {}
+
+  bool SEqualReduce(const ObjectRef& lhs, const ObjectRef& rhs, bool map_free_vars) final {
+    // We cannot use check lhs.same_as(rhs) to check equality.
+    // if we choose to enable var remapping.
+    //
+    // Counter example below (%x, %y) are shared vars
+    // between the two functions(possibly before/after rewriting).
+    //
+    // - function0: fn (%x, %y) { %x + %y }
+    // - function1. fn (%y, %x) { %x + %y }
+    //
+    // Because we choose to enable var remapping,
+    // %x is mapped to %y, and %y is mapped to %x,
+    // the body of the function no longer means the same thing.
+    //
+    // Take away: We can either choose only compare Var by address,
+    // in which case we can use same_as for quick checking,
+    // or we have to run deep comparison and avoid to use same_as checks.
+    auto run = [=]() {
+      if (!lhs.defined() && !rhs.defined()) return true;
 
 Review comment:
   You are indeed right. In this case, we might need to do a special check in Equal to check the both null case. If the call appears in the SEqualReduce then we are fine(the behavior is correct)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-605109113
 
 
   Just did an update to distinguish between graph node(needed in relay semantics) and normal nodes. Please take another look at the definition of structural equality as in comment and see if it makes sense to you.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399432751
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   This doesnt seems very right to me. What if vx has effect? 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399330782
 
 

 ##########
 File path: include/tvm/node/structural_equal.h
 ##########
 @@ -0,0 +1,210 @@
+/*
+ * 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 tvm/node/structural_equal.h
+ * \brief Printer class to print repr string of each AST/IR nodes.
+ */
+#ifndef TVM_NODE_STRUCTURAL_EQUAL_H_
+#define TVM_NODE_STRUCTURAL_EQUAL_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/container.h>
+#include <string>
+
+namespace tvm {
+
+/*!
+ * \brief Equality definition of base value class.
+ */
+class BaseValueEqual {
+ public:
+  bool operator()(const double& lhs, const double& rhs) const {
 
 Review comment:
   likely covered by double already

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399481397
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   note that zx are short hand for `(vx+vx)` here, they are not separate variable. So the comparison  here is a comparing a diamond dag of (x+x) * (x+x) vs a tree. In the case of TIR both are simply treated as a tree atm

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] MarisaKirisame commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604819014
 
 
   Sorry, didnt see that.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399434631
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   Note that the main point of difference is whether we treat a node as graph node or normal one. the current semantics on the TIR is to treat expr nodes as normal nodes. We could change the definition by marking all TIR expr as graph nodes as well(if we decided to do so later), which will give the other behavior.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399481397
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   note that zx are short hand for `(vx+vx)` here (need to read the entire function), see L53,  they are not separate variable. So the comparison  here is a comparing a diamond dag of (x+x) * (x+x) vs a tree. In the case of TIR(where graph node semantics is not yet applied) both are simply treated as a tree atm

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] junrushao1994 commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
junrushao1994 commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604838200
 
 
   Will do tomorrow

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen edited a comment on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604783300
 
 
   cc @jroesch @MarisaKirisame @zhiics @wweic @yzhliu @Hzfengsy @merrymercy 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen merged pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen merged pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399839877
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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 src/node/structural_equal.cc
+ */
+#include <tvm/node/structural_equal.h>
+#include <tvm/node/reflection.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/node.h>
+#include <tvm/runtime/registry.h>
+
+#include <unordered_map>
+
+namespace tvm {
+
+// Define the dispatch functio here since primary user is in this file.
+bool ReflectionVTable::
+SEqualReduce(const Object* self, const Object* other, SEqualReducer equal) const {
+  uint32_t tindex = self->type_index();
+  if (tindex >= fsequal_.size() || fsequal_[tindex] == nullptr) {
+    LOG(FATAL) << "TypeError: SEqualReduce of " << self->GetTypeKey()
+        << " is not registered via TVM_REGISTER_NODE_TYPE";
+  }
+  return fsequal_[tindex](self, other, equal);
+}
+
+/*!
+ * \brief A non recursive stack based SEqual handler that can remaps vars.
+ *
+ *  This handler pushs the Object equality cases into a stack, and
+ *  traverses the stack to expand the necessary children that need to be checked.
+ *
+ *  The order of SEqual being called is the same as the order as if we
+ *  eagerly do recursive calls in SEqualReduce.
+ */
+class RemapVarSEqualHandler :
+      public SEqualReducer::Handler {
+ public:
+  explicit RemapVarSEqualHandler(bool assert_mode)
+      : assert_mode_(assert_mode) {}
+
+  bool SEqualReduce(const ObjectRef& lhs, const ObjectRef& rhs, bool map_free_vars) final {
+    // We cannot use check lhs.same_as(rhs) to check equality.
+    // if we choose to enable var remapping.
+    //
+    // Counter example below (%x, %y) are shared vars
+    // between the two functions(possibly before/after rewriting).
+    //
+    // - function0: fn (%x, %y) { %x + %y }
+    // - function1. fn (%y, %x) { %x + %y }
+    //
+    // Because we choose to enable var remapping,
+    // %x is mapped to %y, and %y is mapped to %x,
+    // the body of the function no longer means the same thing.
+    //
+    // Take away: We can either choose only compare Var by address,
+    // in which case we can use same_as for quick checking,
+    // or we have to run deep comparison and avoid to use same_as checks.
+    auto run = [=]() {
+      if (!lhs.defined() && !rhs.defined()) return true;
 
 Review comment:
   @tqchen I am not sure, but I feel like there might be a bug here when both `lhs` and `rhs` are not defined, because we wont execute Line86. However Line 111 expects L86 is executed. Am I right?
   
   If this is true, we may want to move Line77 before Line106, or we do `emplace_back(ObjectRef(nullptr), ...)` before `return true` and `return true` when both `lhs` and `rhs` are not defined.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399052792
 
 

 ##########
 File path: include/tvm/node/structural_equal.h
 ##########
 @@ -0,0 +1,210 @@
+/*
+ * 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 tvm/node/structural_equal.h
+ * \brief Printer class to print repr string of each AST/IR nodes.
+ */
+#ifndef TVM_NODE_STRUCTURAL_EQUAL_H_
+#define TVM_NODE_STRUCTURAL_EQUAL_H_
+
+#include <tvm/runtime/data_type.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/container.h>
+#include <string>
+
+namespace tvm {
+
+/*!
+ * \brief Equality definition of base value class.
+ */
+class BaseValueEqual {
+ public:
+  bool operator()(const double& lhs, const double& rhs) const {
 
 Review comment:
   do we need float as well?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen edited a comment on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604783300
 
 
   cc @jroesch @MarisaKirisame @zhiics @wweic @yzhliu @Hzfengsy @merrymercy @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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399481397
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   note that zx are short hand for `(vx+vx)` here, they are not separate variable. So the comparison  here is a comparing a diamond dag of (x+x) * (x+x) vs a tree. In the case of TIR(where graph node semantics is not yet applied) both are simply treated as a tree atm

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399450430
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   I'm also confused by this one, I don't see the argument for why these are ever equal.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399839877
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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 src/node/structural_equal.cc
+ */
+#include <tvm/node/structural_equal.h>
+#include <tvm/node/reflection.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/node.h>
+#include <tvm/runtime/registry.h>
+
+#include <unordered_map>
+
+namespace tvm {
+
+// Define the dispatch functio here since primary user is in this file.
+bool ReflectionVTable::
+SEqualReduce(const Object* self, const Object* other, SEqualReducer equal) const {
+  uint32_t tindex = self->type_index();
+  if (tindex >= fsequal_.size() || fsequal_[tindex] == nullptr) {
+    LOG(FATAL) << "TypeError: SEqualReduce of " << self->GetTypeKey()
+        << " is not registered via TVM_REGISTER_NODE_TYPE";
+  }
+  return fsequal_[tindex](self, other, equal);
+}
+
+/*!
+ * \brief A non recursive stack based SEqual handler that can remaps vars.
+ *
+ *  This handler pushs the Object equality cases into a stack, and
+ *  traverses the stack to expand the necessary children that need to be checked.
+ *
+ *  The order of SEqual being called is the same as the order as if we
+ *  eagerly do recursive calls in SEqualReduce.
+ */
+class RemapVarSEqualHandler :
+      public SEqualReducer::Handler {
+ public:
+  explicit RemapVarSEqualHandler(bool assert_mode)
+      : assert_mode_(assert_mode) {}
+
+  bool SEqualReduce(const ObjectRef& lhs, const ObjectRef& rhs, bool map_free_vars) final {
+    // We cannot use check lhs.same_as(rhs) to check equality.
+    // if we choose to enable var remapping.
+    //
+    // Counter example below (%x, %y) are shared vars
+    // between the two functions(possibly before/after rewriting).
+    //
+    // - function0: fn (%x, %y) { %x + %y }
+    // - function1. fn (%y, %x) { %x + %y }
+    //
+    // Because we choose to enable var remapping,
+    // %x is mapped to %y, and %y is mapped to %x,
+    // the body of the function no longer means the same thing.
+    //
+    // Take away: We can either choose only compare Var by address,
+    // in which case we can use same_as for quick checking,
+    // or we have to run deep comparison and avoid to use same_as checks.
+    auto run = [=]() {
+      if (!lhs.defined() && !rhs.defined()) return true;
 
 Review comment:
   @tqchen I am not sure, but I feel like there might be a bug here when both `lhs` and `rhs` are not defined, because we wont execute Line86. However Line 111 expects L86 is executed. Am I right?
   
   If this is true, we may want to move Line77 before Line106, or we do `emplace_back(ObjectRef(nullptr), ...)` before `return true` and `return true` at Line178 when both `lhs` and `rhs` are not defined.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604816448
 
 
   @MarisaKirisame what you asked is supported by the assert_mode_ flag 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399839877
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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 src/node/structural_equal.cc
+ */
+#include <tvm/node/structural_equal.h>
+#include <tvm/node/reflection.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/node.h>
+#include <tvm/runtime/registry.h>
+
+#include <unordered_map>
+
+namespace tvm {
+
+// Define the dispatch functio here since primary user is in this file.
+bool ReflectionVTable::
+SEqualReduce(const Object* self, const Object* other, SEqualReducer equal) const {
+  uint32_t tindex = self->type_index();
+  if (tindex >= fsequal_.size() || fsequal_[tindex] == nullptr) {
+    LOG(FATAL) << "TypeError: SEqualReduce of " << self->GetTypeKey()
+        << " is not registered via TVM_REGISTER_NODE_TYPE";
+  }
+  return fsequal_[tindex](self, other, equal);
+}
+
+/*!
+ * \brief A non recursive stack based SEqual handler that can remaps vars.
+ *
+ *  This handler pushs the Object equality cases into a stack, and
+ *  traverses the stack to expand the necessary children that need to be checked.
+ *
+ *  The order of SEqual being called is the same as the order as if we
+ *  eagerly do recursive calls in SEqualReduce.
+ */
+class RemapVarSEqualHandler :
+      public SEqualReducer::Handler {
+ public:
+  explicit RemapVarSEqualHandler(bool assert_mode)
+      : assert_mode_(assert_mode) {}
+
+  bool SEqualReduce(const ObjectRef& lhs, const ObjectRef& rhs, bool map_free_vars) final {
+    // We cannot use check lhs.same_as(rhs) to check equality.
+    // if we choose to enable var remapping.
+    //
+    // Counter example below (%x, %y) are shared vars
+    // between the two functions(possibly before/after rewriting).
+    //
+    // - function0: fn (%x, %y) { %x + %y }
+    // - function1. fn (%y, %x) { %x + %y }
+    //
+    // Because we choose to enable var remapping,
+    // %x is mapped to %y, and %y is mapped to %x,
+    // the body of the function no longer means the same thing.
+    //
+    // Take away: We can either choose only compare Var by address,
+    // in which case we can use same_as for quick checking,
+    // or we have to run deep comparison and avoid to use same_as checks.
+    auto run = [=]() {
+      if (!lhs.defined() && !rhs.defined()) return true;
 
 Review comment:
   @tqchen I am not sure, but I feel like there might be a bug here when both `lhs` and `rhs` are not defined, because we wont execute Line86. However Line 111 expects L86 is executed. Am I right?
   
   If this true, we may want to move Line77 before Line106, or we do `emplace_back(ObjectRef(nullptr), ...)` before `return true` and `return true` when both `lhs` and `rhs` are not defined.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399416785
 
 

 ##########
 File path: include/tvm/relay/expr.h
 ##########
 @@ -107,7 +107,13 @@ class TupleNode : public ExprNode {
   }
 
   bool SEqualReduce(const TupleNode* other, SEqualReducer equal) const {
-    return equal(fields, other->fields);
+    // specially handle empty tuple as constant and not a graph node.
 
 Review comment:
   a/and/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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
zhiics commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399854886
 
 

 ##########
 File path: src/node/structural_equal.cc
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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 src/node/structural_equal.cc
+ */
+#include <tvm/node/structural_equal.h>
+#include <tvm/node/reflection.h>
+#include <tvm/node/functor.h>
+#include <tvm/node/node.h>
+#include <tvm/runtime/registry.h>
+
+#include <unordered_map>
+
+namespace tvm {
+
+// Define the dispatch functio here since primary user is in this file.
+bool ReflectionVTable::
+SEqualReduce(const Object* self, const Object* other, SEqualReducer equal) const {
+  uint32_t tindex = self->type_index();
+  if (tindex >= fsequal_.size() || fsequal_[tindex] == nullptr) {
+    LOG(FATAL) << "TypeError: SEqualReduce of " << self->GetTypeKey()
+        << " is not registered via TVM_REGISTER_NODE_TYPE";
+  }
+  return fsequal_[tindex](self, other, equal);
+}
+
+/*!
+ * \brief A non recursive stack based SEqual handler that can remaps vars.
+ *
+ *  This handler pushs the Object equality cases into a stack, and
+ *  traverses the stack to expand the necessary children that need to be checked.
+ *
+ *  The order of SEqual being called is the same as the order as if we
+ *  eagerly do recursive calls in SEqualReduce.
+ */
+class RemapVarSEqualHandler :
+      public SEqualReducer::Handler {
+ public:
+  explicit RemapVarSEqualHandler(bool assert_mode)
+      : assert_mode_(assert_mode) {}
+
+  bool SEqualReduce(const ObjectRef& lhs, const ObjectRef& rhs, bool map_free_vars) final {
+    // We cannot use check lhs.same_as(rhs) to check equality.
+    // if we choose to enable var remapping.
+    //
+    // Counter example below (%x, %y) are shared vars
+    // between the two functions(possibly before/after rewriting).
+    //
+    // - function0: fn (%x, %y) { %x + %y }
+    // - function1. fn (%y, %x) { %x + %y }
+    //
+    // Because we choose to enable var remapping,
+    // %x is mapped to %y, and %y is mapped to %x,
+    // the body of the function no longer means the same thing.
+    //
+    // Take away: We can either choose only compare Var by address,
+    // in which case we can use same_as for quick checking,
+    // or we have to run deep comparison and avoid to use same_as checks.
+    auto run = [=]() {
+      if (!lhs.defined() && !rhs.defined()) return true;
 
 Review comment:
   Thanks. I will send it together with the PR that removes `AttrEqual` to reduce the burden of CI.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-605049843
 
 
   re : `_type_has_method_sequal_reduce` is a static variable indicating whether the method is available, and is used by the reflection vtable registration see reflection.h

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399439011
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   BTW, in this case, vx is a variable, so this line alone won't have any effect(other statements that generates bindings to vx might have effects)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on issue #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#issuecomment-604783300
 
 
   cc @jroesch @MarisaKirisame @zhiics @wweic @yzhliu @Hzfengsy 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5154: [NODE][IR] Introduce StructuralEqual Infra for the unified IR.
URL: https://github.com/apache/incubator-tvm/pull/5154#discussion_r399434059
 
 

 ##########
 File path: tests/python/unittest/test_tir_structural_equal.py
 ##########
 @@ -55,8 +55,8 @@ def test_exprs():
     assert tvm.ir.structural_equal(zx * zx, zx * zx)
     assert tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=True)
     assert not tvm.ir.structural_equal(zx * zx, zy * zy, map_free_vars=False)
-    assert not tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
-                                       map_free_vars=False)
+    assert tvm.ir.structural_equal(zx * zx, (vx + vx) * (vx + vx),
 
 Review comment:
   if vx has effect, then the effect itself should be a graph node(in the case of zx) and the equality will no longer holds.

----------------------------------------------------------------
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


With regards,
Apache Git Services