You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/10/22 23:59:31 UTC

[GitHub] [tvm] anwang2009 opened a new pull request #9357: Draft: Add pass to fold type transformations into function signature

anwang2009 opened a new pull request #9357:
URL: https://github.com/apache/tvm/pull/9357


   As title.
   
   This transformer gives users more control over where preprocessing steps may be located, and what input types their relay program can accept


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

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

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



[GitHub] [tvm] anwang2009 commented on a change in pull request #9357: Draft: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.

Review comment:
       Multiple function is weird because you have stuff like
   
   ```
   fn %f1(%in: Tensor(dtype=float32) {
     cast(%in, dtype="float16")
     ...
   }
   
   fn %f2(%in: Tensor(dtype=float32) {
     %0 = add(%in, %in)
     f1(%0)
   }
   ```
   and then we need to translate the result to
   
   ```
   fn %f1(%in: Tensor(dtype=float16) {
     ...
   }
   
   fn %f2(%in: Tensor(dtype=float32) {
     %0 = add(%in, %in)
     %1 = cast(%0)
     f1(%1)
   }
   ```
   
   which is just a lot more steps. 
   
   Most quantized or to-mixed-precision passed models are single function so unless there is a greater need for multi function transformations I'm hoping to leave this as a simple transformation in the first iteration.
   

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+

Review comment:
       Ah, the message isn't about the function being called twice, it's about the param being directly fed into different operations, like 
   
   ```
        in
      /     \
    cast  add
   ```
   This is a case I didn't want to handle in the initial iteration because it's gnarly and uncommon. I just clarified the pass-level comment to this effect.

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "FoldTypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);

Review comment:
       It's required to remove the cast/quantize call, otherwise if the call is returned those operators are persisted in the graph.




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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+

Review comment:
       OK, makes 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.

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

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



[GitHub] [tvm] jroesch commented on pull request #9357: Draft: Add pass to fold type transformations into function signature

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


   cc @electriclilies can you review this one?


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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "LiftDtypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function parameter";
+        }
+      }
+    }
+
+    return Call(cur_op, post_call_node->args, pre_call_node->attrs, pre_call_node->type_args,
+                pre_call_node->span);
+  }
+
+  Expr VisitExpr_(const FunctionNode* node) {
+    function_count_++;
+    if (function_count_ > 1) {
+      CHECK(false) << "LiftDtypeTransformation is supported for only single-function graphs";
+    }
+
+    for (auto param : node->params) {
+      input_transform_map_.insert(std::pair<Var, const CallNode*>(param, NULL));
+    }
+    auto body = this->Mutate(node->body);
+
+    return Function(node->params, body, node->ret_type, node->type_params, node->attrs, node->span);
+  }
+
+  const Op cast_op_ = Op::Get("cast");
+  const Op quantize_op_ = Op::Get("qnn.quantize");
+
+ private:
+  // Maps function parameter to the first-encountered call node within
+  // the function that takes in that input.
+  std::map<Var, const CallNode*> input_transform_map_;
+
+  // Tracks number of functions in this program.
+  int function_count_;
+};
+
+Expr LiftDtypeTransformation(const Expr& expr, const IRModule& mod) {
+  return LiftDtypeTransformationRewriter().Mutate(expr);
+}
+
+namespace transform {
+
+Pass LiftDtypeTransformation() {
+  runtime::TypedPackedFunc<Function(Function, IRModule, PassContext)> pass_func =
+      [=](Function f, IRModule m, PassContext pc) {
+        return Downcast<Function>(LiftDtypeTransformation(f, m));
+      };
+  return CreateFunctionPass(pass_func, 0, "LiftDtypeTransformation", {});

Review comment:
       I think you probably want the opt-level to be higher here, probably 4, so that the user needs to specify it before it is run. 




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

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

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



[GitHub] [tvm] anwang2009 commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: tests/python/relay/test_fold_type_transformation.py
##########
@@ -0,0 +1,51 @@
+# 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
+
+
+def test_simple_cast_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.cast(data, "float16")
+    out = relay.add(out, out)
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="float16")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)
+
+
+def test_simple_quantize_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.qnn.op.quantize(data, relay.const(2.0), relay.const(0), out_dtype="uint8")
+    out = relay.add(out, out)
+
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="uint8")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)

Review comment:
       Turns out the irmod creation automatically wraps the expressions in a function, resolved




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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "LiftDtypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function parameter";
+        }

Review comment:
       Do the arguments to the call and the function parameters need to be the same global vars?

##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "LiftDtypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function parameter";
+        }
+      }
+    }
+
+    return Call(cur_op, post_call_node->args, pre_call_node->attrs, pre_call_node->type_args,
+                pre_call_node->span);
+  }
+
+  Expr VisitExpr_(const FunctionNode* node) {
+    function_count_++;
+    if (function_count_ > 1) {
+      CHECK(false) << "LiftDtypeTransformation is supported for only single-function graphs";
+    }
+
+    for (auto param : node->params) {
+      input_transform_map_.insert(std::pair<Var, const CallNode*>(param, NULL));
+    }

Review comment:
       Do you need to rewrite the function parameters 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.

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: tests/python/relay/test_fold_type_transformation.py
##########
@@ -0,0 +1,51 @@
+# 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
+
+
+def test_simple_cast_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.cast(data, "float16")
+    out = relay.add(out, out)
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="float16")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)
+
+
+def test_simple_quantize_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.qnn.op.quantize(data, relay.const(2.0), relay.const(0), out_dtype="uint8")
+    out = relay.add(out, out)
+
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="uint8")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)

Review comment:
       Yeah, right now your tests don't actually wrap the dataflow in a function




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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "FoldTypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);

Review comment:
       Oh, got it, misread what was going on 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.

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";

Review comment:
       I think this is actually an argument to the call, not a parameter in this case. 

##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "LiftDtypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function parameter";
+        }

Review comment:
       Do the arguments to the call and the function parameters need to be the same global vars?

##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "LiftDtypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function parameter";
+        }
+      }
+    }
+
+    return Call(cur_op, post_call_node->args, pre_call_node->attrs, pre_call_node->type_args,
+                pre_call_node->span);
+  }
+
+  Expr VisitExpr_(const FunctionNode* node) {
+    function_count_++;
+    if (function_count_ > 1) {
+      CHECK(false) << "LiftDtypeTransformation is supported for only single-function graphs";
+    }
+
+    for (auto param : node->params) {
+      input_transform_map_.insert(std::pair<Var, const CallNode*>(param, NULL));
+    }

Review comment:
       Do you need to rewrite the function parameters 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.

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

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



[GitHub] [tvm] AndrewZhaoLuo commented on pull request #9357: Add pass to fold type transformations into function signature

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


   @anwang2009 do you still have interest in merging this 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.

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

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



[GitHub] [tvm] electriclilies commented on pull request #9357: Add pass to fold type transformations into function signature

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


   After taking another look, I'm a bit confused how you're doing the rewrite of the function parameters, added some questions related to 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.

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

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



[GitHub] [tvm] masahi commented on pull request #9357: Add pass to fold type transformations into function signature

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


   @anwang2009 Please take a look at the CI problem.


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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Draft: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.

Review comment:
       Not sure what this means, can you clarify? (Is it that you shouldn't run the pass on a function multiple times?)

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "FoldTypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function-level input";
+        }
+      }
+    }
+
+    return Call(cur_op, post_call_node->args, pre_call_node->attrs, pre_call_node->type_args,
+                pre_call_node->span);
+  }
+
+  Expr VisitExpr_(const FunctionNode* node) {
+    function_count_++;
+    if (function_count_ > 1) {
+      CHECK(false) << "FoldTypeTransformation is supported for only single-function graphs";
+    }
+
+    for (auto param : node->params) {
+      input_transform_map_.insert(std::pair<Var, const CallNode*>(param, NULL));
+    }
+    auto body = this->Mutate(node->body);
+
+    return Function(node->params, body, node->ret_type, node->type_params, node->attrs, node->span);
+  }
+
+  const Op cast_op_ = Op::Get("cast");
+  const Op quantize_op_ = Op::Get("qnn.quantize");
+
+ private:
+  // Maps function-level input to the first-encountered call node within
+  // the function that takes in that input.
+  std::map<Var, const CallNode*> input_transform_map_;
+
+  // Tracks number of functions in this program.
+  int function_count_;
+};
+
+Expr FoldTypeTransformation(const Expr& expr, const IRModule& mod) {

Review comment:
       I think technically you are lifting the cast and quantize ops out of the function -- something like "LiftDtypeTransformation" might be a better name

##########
File path: tests/python/relay/test_fold_type_transformation.py
##########
@@ -0,0 +1,51 @@
+# 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
+
+
+def test_simple_cast_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.cast(data, "float16")
+    out = relay.add(out, out)
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="float16")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)
+
+
+def test_simple_quantize_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.qnn.op.quantize(data, relay.const(2.0), relay.const(0), out_dtype="uint8")
+    out = relay.add(out, out)
+
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="uint8")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)

Review comment:
       Could you add a test where the call's op is a relay function, not just an op? 

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "FoldTypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);

Review comment:
       correct me if I'm wrong, but it seems like you would want to return a call from this rewrite function in all cases? I think you don't need this return at all.

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is

Review comment:
       nit: it seems like you are using "function-level input" to refer to function arguments and parameters, I think it would help readability to change it to use those terms instead.

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function-level input var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function input with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+

Review comment:
       Why can't we do the transform if the function is called twice? Could you just turn the map into a counter and not transform the arguments if you've already seen them?

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.

Review comment:
       Why can this pass only run on modules containing a single function? It would be great if it could generalize to modules containing more than one function.

##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type of each function-level input is transformed only once
+ *     per program.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ */
+class FoldTypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    CHECK(post_call_node) << "Expected a CallNode, but got " << post;
+

Review comment:
       I think this should be an ICHECK, not a CHECK since it is an internal error, not a user facing one. 

##########
File path: python/tvm/relay/transform/transform.py
##########
@@ -1248,3 +1248,10 @@ def SplitArgs(max_function_args):
         The registered pass for constant folding.
     """
     return _ffi_api.SplitArgs(max_function_args)
+
+
+def FoldTypeTransformation():
+    """
+    Automatic function signature transformation

Review comment:
       Can you add detail about what the transformation does




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

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

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



[GitHub] [tvm] anwang2009 commented on a change in pull request #9357: Draft: Add pass to fold type transformations into function signature

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



##########
File path: tests/python/relay/test_fold_type_transformation.py
##########
@@ -0,0 +1,51 @@
+# 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
+
+
+def test_simple_cast_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.cast(data, "float16")
+    out = relay.add(out, out)
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="float16")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)
+
+
+def test_simple_quantize_fold():
+    data = relay.var("data", shape=[1, 3, 224, 224], dtype="float32")
+    out = relay.qnn.op.quantize(data, relay.const(2.0), relay.const(0), out_dtype="uint8")
+    out = relay.add(out, out)
+
+    mod = tvm.IRModule.from_expr(out)
+    mod = tvm.relay.transform.InferType()(mod)
+    mod = tvm.relay.transform.FoldTypeTransformation()(mod)
+
+    data_fp16 = relay.var("data", shape=[1, 3, 224, 224], dtype="uint8")
+    out = relay.add(data_fp16, data_fp16)
+    expected_mod = tvm.IRModule.from_expr(out)
+    expected_mod = tvm.relay.transform.InferType()(expected_mod)
+
+    assert tvm.ir.structural_equal(mod, expected_mod)

Review comment:
       Do you mean like 
   
   ```
   fn(in) {
     %0 = cast(in)
     %1 = other_fn(%0)
   }
   ```
   or something else?




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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/lift_dtype_transformation.cc
##########
@@ -0,0 +1,167 @@
+/*
+ * 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/relay/transforms/lift_dtype_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function parameter is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function parameter is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.
+ *   - The type transformation operation must be either a "cast"
+ *     or "qnn.quantize".
+ *   - Each function parameter is used only once
+ *     per program. There should be no structure that looks like:
+ * 
+ *      in                                      in
+ *     /  \        but the following is ok:      |
+ *  cast  add                                  cast
+ */
+class LiftDtypeTransformationRewriter : public MixedModeMutator {
+ protected:
+  Expr Rewrite_(const CallNode* pre_call_node, const Expr& post) final {
+    const CallNode* post_call_node = post.as<CallNode>();
+    ICHECK(post_call_node) << "Expected a CallNode, but got " << post;
+
+    Expr cur_op = pre_call_node->op;
+    for (auto arg : pre_call_node->args) {
+      auto maybe_var_node = arg.as<VarNode>();
+      if (maybe_var_node) {
+        auto var = Downcast<Var>(arg);
+        auto it = input_transform_map_.find(var);
+        if (it != input_transform_map_.end()) {
+          // Checks that the function parameter var hasn't been an arg
+          // to a CallNode yet.
+          CHECK(!it->second) << "Function param with name '" << var->name_hint()
+                             << "' is fed into more than one call; "
+                             << "aborting transformation";
+
+          it->second = pre_call_node;
+
+          // Get the type to transform the function signature to
+          DataType out_dtype;
+          if (cur_op == cast_op_) {
+            auto attrs = pre_call_node->attrs.as<CastAttrs>();
+            out_dtype = attrs->dtype;
+          } else if (cur_op == quantize_op_) {
+            auto attrs = pre_call_node->attrs.as<qnn::QuantizeAttrs>();
+            out_dtype = attrs->out_dtype;
+          } else {
+            CHECK(false) << "LiftDtypeTransformation will only fold cast and "
+                         << "quantize type transformations";
+          }
+
+          // Mutate the var node type
+          VarNode* var_node = const_cast<VarNode*>(maybe_var_node);
+          const TensorTypeNode* anno = var_node->type_annotation.as<TensorTypeNode>();
+          auto mut_anno = const_cast<TensorTypeNode*>(anno);
+          auto shape = anno->shape;
+          mut_anno->dtype = out_dtype;
+
+          return GetRef<Expr>(var_node);
+        } else {
+          LOG(WARNING) << "Variable '" << var->name_hint() << "' encountered"
+                       << " but wasn't registered as a function parameter";
+        }
+      }
+    }
+
+    return Call(cur_op, post_call_node->args, pre_call_node->attrs, pre_call_node->type_args,
+                pre_call_node->span);
+  }
+
+  Expr VisitExpr_(const FunctionNode* node) {
+    function_count_++;
+    if (function_count_ > 1) {
+      CHECK(false) << "LiftDtypeTransformation is supported for only single-function graphs";
+    }
+
+    for (auto param : node->params) {
+      input_transform_map_.insert(std::pair<Var, const CallNode*>(param, NULL));
+    }
+    auto body = this->Mutate(node->body);
+
+    return Function(node->params, body, node->ret_type, node->type_params, node->attrs, node->span);
+  }
+
+  const Op cast_op_ = Op::Get("cast");
+  const Op quantize_op_ = Op::Get("qnn.quantize");
+
+ private:
+  // Maps function parameter to the first-encountered call node within
+  // the function that takes in that input.
+  std::map<Var, const CallNode*> input_transform_map_;
+
+  // Tracks number of functions in this program.
+  int function_count_;
+};
+
+Expr LiftDtypeTransformation(const Expr& expr, const IRModule& mod) {
+  return LiftDtypeTransformationRewriter().Mutate(expr);
+}
+
+namespace transform {
+
+Pass LiftDtypeTransformation() {
+  runtime::TypedPackedFunc<Function(Function, IRModule, PassContext)> pass_func =
+      [=](Function f, IRModule m, PassContext pc) {
+        return Downcast<Function>(LiftDtypeTransformation(f, m));
+      };
+  return CreateFunctionPass(pass_func, 0, "LiftDtypeTransformation", {});

Review comment:
       I think you probably want the opt-level to be higher here, probably 3 or 4, so that the user needs to specify it before it is run. 




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

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

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



[GitHub] [tvm] electriclilies commented on a change in pull request #9357: Add pass to fold type transformations into function signature

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



##########
File path: src/relay/transforms/fold_type_transformation.cc
##########
@@ -0,0 +1,163 @@
+/*
+ * 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/relay/transforms/fold_type_transformation.cc
+ * \brief A pass for transforming relay graph function
+ * signatures such that when a function-level inputs is
+ * transformed by a subsequent cast or quantize operation,
+ * that operation is folded into the signature itself.
+ */
+
+#include <tvm/relay/expr.h>
+#include <tvm/relay/expr_functor.h>
+#include <tvm/relay/qnn/attrs.h>
+#include <tvm/relay/transform.h>
+
+namespace tvm {
+namespace relay {
+
+/*! \brief This class transforms a relay module's function signature
+ * such that when a function-level input is transformed by a subsequent
+ * "cast" or "qnn.quantize" operation, that operation is folded into
+ * the signature itself. For example,
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), float32]) {
+ *   %0 = qnn.quantize(%data, 2f, 0, out_dtype="uint8");
+ *   add(%0, %0)
+ * }
+ *
+ * would be transformed to
+ *
+ * def @main(%data: Tensor[(1, 3, 224, 224), uint8]) {
+ *   add(%0, %0)
+ * }
+ *
+ * Note that now it is the user's responsibility to modify their
+ * input pre-processing pipeline to satisfy the new signature's
+ * constraints.
+ *
+ * For this pass to fold a type transformation, the following conditions
+ * must be met:
+ *   - The relay module must contain only a single function.

Review comment:
       Alright, this is OK for the first iteration




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

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

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