You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by "spectrometerHBH (via GitHub)" <gi...@apache.org> on 2023/03/08 18:15:47 UTC

[GitHub] [tvm] spectrometerHBH opened a new pull request, #14242: [Unity][Transform] Automatic Mixed Precision

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

   This PR adds a new pass ToMixedPrecision to automatically cast fp32 models to fp16 when necessary.


-- 
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] spectrometerHBH commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "spectrometerHBH (via GitHub)" <gi...@apache.org>.
spectrometerHBH commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1142612682


##########
tests/python/relax/test_transform_to_mixed_precision.py:
##########
@@ -0,0 +1,540 @@
+# 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 numpy as np
+import tvm
+from tvm import relax
+import tvm.testing
+from tvm.relax.transform import ToMixedPrecision
+from tvm.script.parser import ir as I, relax as R
+
+
+def _assert_test(input, expected):
+    mod = ToMixedPrecision()(input)
+    tvm.ir.assert_structural_equal(mod, expected)
+
+
+def test_conv2d():
+    @I.ir_module
+    class Input:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), "float32"), w: R.Tensor((4, 3, 3, 3), "float32")
+        ) -> R.Tensor(None, "float32", ndim=4):
+            with R.dataflow():
+                gv: R.Tensor((2, 4, 26, 26), "float32") = R.nn.conv2d(x, w, out_dtype="float32")
+                R.output(gv)
+            return gv
+
+    @I.ir_module
+    class Expected:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), dtype="float32"), w: R.Tensor((4, 3, 3, 3), dtype="float32")
+        ) -> R.Tensor((2, 4, 26, 26), dtype="float32"):
+            with R.dataflow():
+                lv: R.Tensor((2, 3, 28, 28), dtype="float16") = R.astype(x, dtype="float16")
+                lv1: R.Tensor((4, 3, 3, 3), dtype="float16") = R.astype(w, dtype="float16")
+                gv: R.Tensor((2, 4, 26, 26), dtype="float32") = R.nn.conv2d(
+                    lv,
+                    lv1,
+                    strides=[1, 1],
+                    padding=[0, 0, 0, 0],
+                    dilation=[1, 1],
+                    groups=1,
+                    data_layout="NCHW",
+                    kernel_layout="OIHW",
+                    out_layout="NCHW",
+                    out_dtype="float32",

Review Comment:
   Sure



-- 
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] yongwww commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "yongwww (via GitHub)" <gi...@apache.org>.
yongwww commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1129891535


##########
src/relax/transform/to_mixed_precision.cc:
##########
@@ -0,0 +1,538 @@
+/*
+ * 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/relax/transform/to_mixed_precision.cc
+ * \brief Automatic mixed precision pass.
+ */
+
+#include <tvm/relax/expr_functor.h>
+#include <tvm/relax/op_attr_types.h>
+#include <tvm/relax/transform.h>
+
+#include <array>
+
+#include "../op/nn/convolution.h"
+#include "../op/tensor/datatype.h"
+#include "../op/tensor/linear_algebra.h"
+#include "infer_amp_utils.h"
+#include "utils.h"
+
+namespace tvm {
+namespace relax {
+
+using runtime::String;
+
+int GetMixedPrecisionInfo(const CallNode* call_node) {
+  const OpNode* op_node = call_node->op.as<OpNode>();
+  if (op_node == nullptr) {
+    return -1;
+  }
+  Op op = GetRef<Op>(op_node);
+  auto attr_map = Op::GetAttrMap<TMixedPrecisionPolicy>("TMixedPrecisionPolicy");
+  return attr_map.count(op) ? attr_map[op] : MixedPrecisionPolicyKind::kNever;
+}
+
+/*!
+ * \brief Main logic to automatically cast fp32 input modules to fp16 for certain ops.
+ *
+ * Structurally speaking, a Relax function is composed of a series of VarBinding and
+ * MatchCast. And a specific class of VarBindings is the basic unit we want to rewrite.
+ * Formally, they are of the form:
+ *
+ * var = Call(Op, [args], attrs)
+ *
+ * where Op is a specific op we want to rewrite, and attrs is the attributes of the op.
+ * var and args are all exprs with type Tensor or Tuple of Tensors. They might
+ * be vars, constants, or Tuple of vars and constants.
+ * Depending on the properties of the op, we may have 3 different ways to rewrite it:
+ *
+ * 1. kAlways: Always cast the args to fp16

Review Comment:
   Probably we will lose some accuracy with `kAlways`, I am wondering if it is possible to allow user to control the `MixedPrecisionPolicyKind` in python side like `ToMixedPrecision(kind=kalways/kfollow/auto/never)`



-- 
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 a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "masahi (via GitHub)" <gi...@apache.org>.
masahi commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1141728784


##########
tests/python/relax/test_transform_to_mixed_precision.py:
##########
@@ -0,0 +1,540 @@
+# 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 numpy as np
+import tvm
+from tvm import relax
+import tvm.testing
+from tvm.relax.transform import ToMixedPrecision
+from tvm.script.parser import ir as I, relax as R
+
+
+def _assert_test(input, expected):
+    mod = ToMixedPrecision()(input)
+    tvm.ir.assert_structural_equal(mod, expected)
+
+
+def test_conv2d():
+    @I.ir_module
+    class Input:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), "float32"), w: R.Tensor((4, 3, 3, 3), "float32")
+        ) -> R.Tensor(None, "float32", ndim=4):
+            with R.dataflow():
+                gv: R.Tensor((2, 4, 26, 26), "float32") = R.nn.conv2d(x, w, out_dtype="float32")
+                R.output(gv)
+            return gv
+
+    @I.ir_module
+    class Expected:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), dtype="float32"), w: R.Tensor((4, 3, 3, 3), dtype="float32")
+        ) -> R.Tensor((2, 4, 26, 26), dtype="float32"):
+            with R.dataflow():
+                lv: R.Tensor((2, 3, 28, 28), dtype="float16") = R.astype(x, dtype="float16")
+                lv1: R.Tensor((4, 3, 3, 3), dtype="float16") = R.astype(w, dtype="float16")
+                gv: R.Tensor((2, 4, 26, 26), dtype="float32") = R.nn.conv2d(
+                    lv,
+                    lv1,
+                    strides=[1, 1],
+                    padding=[0, 0, 0, 0],
+                    dilation=[1, 1],
+                    groups=1,
+                    data_layout="NCHW",
+                    kernel_layout="OIHW",
+                    out_layout="NCHW",
+                    out_dtype="float32",

Review Comment:
   What's the reasoning for always fixing the output dtype to be fp32? This is different from the Relay pass behavior. 



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

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

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


[GitHub] [tvm] spectrometerHBH commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "spectrometerHBH (via GitHub)" <gi...@apache.org>.
spectrometerHBH commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1129930876


##########
src/relax/transform/to_mixed_precision.cc:
##########
@@ -0,0 +1,538 @@
+/*
+ * 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/relax/transform/to_mixed_precision.cc
+ * \brief Automatic mixed precision pass.
+ */
+
+#include <tvm/relax/expr_functor.h>
+#include <tvm/relax/op_attr_types.h>
+#include <tvm/relax/transform.h>
+
+#include <array>
+
+#include "../op/nn/convolution.h"
+#include "../op/tensor/datatype.h"
+#include "../op/tensor/linear_algebra.h"
+#include "infer_amp_utils.h"
+#include "utils.h"
+
+namespace tvm {
+namespace relax {
+
+using runtime::String;
+
+int GetMixedPrecisionInfo(const CallNode* call_node) {
+  const OpNode* op_node = call_node->op.as<OpNode>();
+  if (op_node == nullptr) {
+    return -1;
+  }
+  Op op = GetRef<Op>(op_node);
+  auto attr_map = Op::GetAttrMap<TMixedPrecisionPolicy>("TMixedPrecisionPolicy");
+  return attr_map.count(op) ? attr_map[op] : MixedPrecisionPolicyKind::kNever;
+}
+
+/*!
+ * \brief Main logic to automatically cast fp32 input modules to fp16 for certain ops.
+ *
+ * Structurally speaking, a Relax function is composed of a series of VarBinding and
+ * MatchCast. And a specific class of VarBindings is the basic unit we want to rewrite.
+ * Formally, they are of the form:
+ *
+ * var = Call(Op, [args], attrs)
+ *
+ * where Op is a specific op we want to rewrite, and attrs is the attributes of the op.
+ * var and args are all exprs with type Tensor or Tuple of Tensors. They might
+ * be vars, constants, or Tuple of vars and constants.
+ * Depending on the properties of the op, we may have 3 different ways to rewrite it:
+ *
+ * 1. kAlways: Always cast the args to fp16

Review Comment:
   That's in principle possible. Here the pass is meant to serve as similar things like `torch.autocast("cuda")`, where it will always cast the inputs of matmul/conv2d to fp16 from fp32.



-- 
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] tvm-bot commented on pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "tvm-bot (via GitHub)" <gi...@apache.org>.
tvm-bot commented on PR #14242:
URL: https://github.com/apache/tvm/pull/14242#issuecomment-1460638439

   <!---bot-comment-->
   
   Thanks for contributing to TVM! Please refer to the contributing guidelines https://tvm.apache.org/docs/contribute/ for useful information and tips. Please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @-ing them in a comment.
   
   <!--bot-comment-ccs-start-->
    * cc @quic-sanirudh <sub>See [#10317](https://github.com/apache/tvm/issues/10317) for details</sub><!--bot-comment-ccs-end-->
   
   <sub>Generated by [tvm-bot](https://github.com/apache/tvm/blob/main/ci/README.md#github-actions)</sub>


-- 
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] yongwww commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "yongwww (via GitHub)" <gi...@apache.org>.
yongwww commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1129891535


##########
src/relax/transform/to_mixed_precision.cc:
##########
@@ -0,0 +1,538 @@
+/*
+ * 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/relax/transform/to_mixed_precision.cc
+ * \brief Automatic mixed precision pass.
+ */
+
+#include <tvm/relax/expr_functor.h>
+#include <tvm/relax/op_attr_types.h>
+#include <tvm/relax/transform.h>
+
+#include <array>
+
+#include "../op/nn/convolution.h"
+#include "../op/tensor/datatype.h"
+#include "../op/tensor/linear_algebra.h"
+#include "infer_amp_utils.h"
+#include "utils.h"
+
+namespace tvm {
+namespace relax {
+
+using runtime::String;
+
+int GetMixedPrecisionInfo(const CallNode* call_node) {
+  const OpNode* op_node = call_node->op.as<OpNode>();
+  if (op_node == nullptr) {
+    return -1;
+  }
+  Op op = GetRef<Op>(op_node);
+  auto attr_map = Op::GetAttrMap<TMixedPrecisionPolicy>("TMixedPrecisionPolicy");
+  return attr_map.count(op) ? attr_map[op] : MixedPrecisionPolicyKind::kNever;
+}
+
+/*!
+ * \brief Main logic to automatically cast fp32 input modules to fp16 for certain ops.
+ *
+ * Structurally speaking, a Relax function is composed of a series of VarBinding and
+ * MatchCast. And a specific class of VarBindings is the basic unit we want to rewrite.
+ * Formally, they are of the form:
+ *
+ * var = Call(Op, [args], attrs)
+ *
+ * where Op is a specific op we want to rewrite, and attrs is the attributes of the op.
+ * var and args are all exprs with type Tensor or Tuple of Tensors. They might
+ * be vars, constants, or Tuple of vars and constants.
+ * Depending on the properties of the op, we may have 3 different ways to rewrite it:
+ *
+ * 1. kAlways: Always cast the args to fp16

Review Comment:
   Probably we will lose some accuracy with `kAlways`, I am wondering if it is possible to allow user to control the `MixedPrecisionPolicyKind` like `ToMixedPrecision(MixedPrecisionPolicyKind=kAlways/kFollow/AMP/Never)`



-- 
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] spectrometerHBH commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "spectrometerHBH (via GitHub)" <gi...@apache.org>.
spectrometerHBH commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1142495125


##########
tests/python/relax/test_transform_to_mixed_precision.py:
##########
@@ -0,0 +1,540 @@
+# 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 numpy as np
+import tvm
+from tvm import relax
+import tvm.testing
+from tvm.relax.transform import ToMixedPrecision
+from tvm.script.parser import ir as I, relax as R
+
+
+def _assert_test(input, expected):
+    mod = ToMixedPrecision()(input)
+    tvm.ir.assert_structural_equal(mod, expected)
+
+
+def test_conv2d():
+    @I.ir_module
+    class Input:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), "float32"), w: R.Tensor((4, 3, 3, 3), "float32")
+        ) -> R.Tensor(None, "float32", ndim=4):
+            with R.dataflow():
+                gv: R.Tensor((2, 4, 26, 26), "float32") = R.nn.conv2d(x, w, out_dtype="float32")
+                R.output(gv)
+            return gv
+
+    @I.ir_module
+    class Expected:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), dtype="float32"), w: R.Tensor((4, 3, 3, 3), dtype="float32")
+        ) -> R.Tensor((2, 4, 26, 26), dtype="float32"):
+            with R.dataflow():
+                lv: R.Tensor((2, 3, 28, 28), dtype="float16") = R.astype(x, dtype="float16")
+                lv1: R.Tensor((4, 3, 3, 3), dtype="float16") = R.astype(w, dtype="float16")
+                gv: R.Tensor((2, 4, 26, 26), dtype="float32") = R.nn.conv2d(
+                    lv,
+                    lv1,
+                    strides=[1, 1],
+                    padding=[0, 0, 0, 0],
+                    dilation=[1, 1],
+                    groups=1,
+                    data_layout="NCHW",
+                    kernel_layout="OIHW",
+                    out_layout="NCHW",
+                    out_dtype="float32",

Review Comment:
   You can configure it by passing out_dtype to ToMixedPrecision()



-- 
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 a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "masahi (via GitHub)" <gi...@apache.org>.
masahi commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1142612246


##########
tests/python/relax/test_transform_to_mixed_precision.py:
##########
@@ -0,0 +1,540 @@
+# 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 numpy as np
+import tvm
+from tvm import relax
+import tvm.testing
+from tvm.relax.transform import ToMixedPrecision
+from tvm.script.parser import ir as I, relax as R
+
+
+def _assert_test(input, expected):
+    mod = ToMixedPrecision()(input)
+    tvm.ir.assert_structural_equal(mod, expected)
+
+
+def test_conv2d():
+    @I.ir_module
+    class Input:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), "float32"), w: R.Tensor((4, 3, 3, 3), "float32")
+        ) -> R.Tensor(None, "float32", ndim=4):
+            with R.dataflow():
+                gv: R.Tensor((2, 4, 26, 26), "float32") = R.nn.conv2d(x, w, out_dtype="float32")
+                R.output(gv)
+            return gv
+
+    @I.ir_module
+    class Expected:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), dtype="float32"), w: R.Tensor((4, 3, 3, 3), dtype="float32")
+        ) -> R.Tensor((2, 4, 26, 26), dtype="float32"):
+            with R.dataflow():
+                lv: R.Tensor((2, 3, 28, 28), dtype="float16") = R.astype(x, dtype="float16")
+                lv1: R.Tensor((4, 3, 3, 3), dtype="float16") = R.astype(w, dtype="float16")
+                gv: R.Tensor((2, 4, 26, 26), dtype="float32") = R.nn.conv2d(
+                    lv,
+                    lv1,
+                    strides=[1, 1],
+                    padding=[0, 0, 0, 0],
+                    dilation=[1, 1],
+                    groups=1,
+                    data_layout="NCHW",
+                    kernel_layout="OIHW",
+                    out_layout="NCHW",
+                    out_dtype="float32",

Review Comment:
   ok but `out_dtype = fp16` case should be tested, since it affects where casts are inserted. 



-- 
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] yongwww commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "yongwww (via GitHub)" <gi...@apache.org>.
yongwww commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1129891535


##########
src/relax/transform/to_mixed_precision.cc:
##########
@@ -0,0 +1,538 @@
+/*
+ * 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/relax/transform/to_mixed_precision.cc
+ * \brief Automatic mixed precision pass.
+ */
+
+#include <tvm/relax/expr_functor.h>
+#include <tvm/relax/op_attr_types.h>
+#include <tvm/relax/transform.h>
+
+#include <array>
+
+#include "../op/nn/convolution.h"
+#include "../op/tensor/datatype.h"
+#include "../op/tensor/linear_algebra.h"
+#include "infer_amp_utils.h"
+#include "utils.h"
+
+namespace tvm {
+namespace relax {
+
+using runtime::String;
+
+int GetMixedPrecisionInfo(const CallNode* call_node) {
+  const OpNode* op_node = call_node->op.as<OpNode>();
+  if (op_node == nullptr) {
+    return -1;
+  }
+  Op op = GetRef<Op>(op_node);
+  auto attr_map = Op::GetAttrMap<TMixedPrecisionPolicy>("TMixedPrecisionPolicy");
+  return attr_map.count(op) ? attr_map[op] : MixedPrecisionPolicyKind::kNever;
+}
+
+/*!
+ * \brief Main logic to automatically cast fp32 input modules to fp16 for certain ops.
+ *
+ * Structurally speaking, a Relax function is composed of a series of VarBinding and
+ * MatchCast. And a specific class of VarBindings is the basic unit we want to rewrite.
+ * Formally, they are of the form:
+ *
+ * var = Call(Op, [args], attrs)
+ *
+ * where Op is a specific op we want to rewrite, and attrs is the attributes of the op.
+ * var and args are all exprs with type Tensor or Tuple of Tensors. They might
+ * be vars, constants, or Tuple of vars and constants.
+ * Depending on the properties of the op, we may have 3 different ways to rewrite it:
+ *
+ * 1. kAlways: Always cast the args to fp16

Review Comment:
   Probably we will lose some accuracy with `kAlways`, I am wondering if it is possible to allow user to control the `MixedPrecisionPolicyKind` in python side ToMixedPrecision(kind=kalways/kfollow/auto/never)



-- 
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] junrushao merged pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "junrushao (via GitHub)" <gi...@apache.org>.
junrushao merged PR #14242:
URL: https://github.com/apache/tvm/pull/14242


-- 
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] yongwww commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "yongwww (via GitHub)" <gi...@apache.org>.
yongwww commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1129891535


##########
src/relax/transform/to_mixed_precision.cc:
##########
@@ -0,0 +1,538 @@
+/*
+ * 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/relax/transform/to_mixed_precision.cc
+ * \brief Automatic mixed precision pass.
+ */
+
+#include <tvm/relax/expr_functor.h>
+#include <tvm/relax/op_attr_types.h>
+#include <tvm/relax/transform.h>
+
+#include <array>
+
+#include "../op/nn/convolution.h"
+#include "../op/tensor/datatype.h"
+#include "../op/tensor/linear_algebra.h"
+#include "infer_amp_utils.h"
+#include "utils.h"
+
+namespace tvm {
+namespace relax {
+
+using runtime::String;
+
+int GetMixedPrecisionInfo(const CallNode* call_node) {
+  const OpNode* op_node = call_node->op.as<OpNode>();
+  if (op_node == nullptr) {
+    return -1;
+  }
+  Op op = GetRef<Op>(op_node);
+  auto attr_map = Op::GetAttrMap<TMixedPrecisionPolicy>("TMixedPrecisionPolicy");
+  return attr_map.count(op) ? attr_map[op] : MixedPrecisionPolicyKind::kNever;
+}
+
+/*!
+ * \brief Main logic to automatically cast fp32 input modules to fp16 for certain ops.
+ *
+ * Structurally speaking, a Relax function is composed of a series of VarBinding and
+ * MatchCast. And a specific class of VarBindings is the basic unit we want to rewrite.
+ * Formally, they are of the form:
+ *
+ * var = Call(Op, [args], attrs)
+ *
+ * where Op is a specific op we want to rewrite, and attrs is the attributes of the op.
+ * var and args are all exprs with type Tensor or Tuple of Tensors. They might
+ * be vars, constants, or Tuple of vars and constants.
+ * Depending on the properties of the op, we may have 3 different ways to rewrite it:
+ *
+ * 1. kAlways: Always cast the args to fp16

Review Comment:
   Probably we will lose some accuracy with `kAlways`, I am wondering if it is possible to allow user to control the `MixedPrecisionPolicyKind` like `ToMixedPrecision(MixedPrecisionPolicyKind=kAlways/kFollow)`



-- 
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] yongwww commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "yongwww (via GitHub)" <gi...@apache.org>.
yongwww commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1129891535


##########
src/relax/transform/to_mixed_precision.cc:
##########
@@ -0,0 +1,538 @@
+/*
+ * 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/relax/transform/to_mixed_precision.cc
+ * \brief Automatic mixed precision pass.
+ */
+
+#include <tvm/relax/expr_functor.h>
+#include <tvm/relax/op_attr_types.h>
+#include <tvm/relax/transform.h>
+
+#include <array>
+
+#include "../op/nn/convolution.h"
+#include "../op/tensor/datatype.h"
+#include "../op/tensor/linear_algebra.h"
+#include "infer_amp_utils.h"
+#include "utils.h"
+
+namespace tvm {
+namespace relax {
+
+using runtime::String;
+
+int GetMixedPrecisionInfo(const CallNode* call_node) {
+  const OpNode* op_node = call_node->op.as<OpNode>();
+  if (op_node == nullptr) {
+    return -1;
+  }
+  Op op = GetRef<Op>(op_node);
+  auto attr_map = Op::GetAttrMap<TMixedPrecisionPolicy>("TMixedPrecisionPolicy");
+  return attr_map.count(op) ? attr_map[op] : MixedPrecisionPolicyKind::kNever;
+}
+
+/*!
+ * \brief Main logic to automatically cast fp32 input modules to fp16 for certain ops.
+ *
+ * Structurally speaking, a Relax function is composed of a series of VarBinding and
+ * MatchCast. And a specific class of VarBindings is the basic unit we want to rewrite.
+ * Formally, they are of the form:
+ *
+ * var = Call(Op, [args], attrs)
+ *
+ * where Op is a specific op we want to rewrite, and attrs is the attributes of the op.
+ * var and args are all exprs with type Tensor or Tuple of Tensors. They might
+ * be vars, constants, or Tuple of vars and constants.
+ * Depending on the properties of the op, we may have 3 different ways to rewrite it:
+ *
+ * 1. kAlways: Always cast the args to fp16

Review Comment:
   Probably we will lose some accuracy with `kAlways`, I am wondering if it is possible to allow user to control the `MixedPrecisionPolicyKind` in python side like `ToMixedPrecision(MixedPrecisionPolicyKind=kAlways/kFollow/AMP/Never)`



-- 
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] spectrometerHBH commented on a diff in pull request #14242: [Unity][Transform] Automatic Mixed Precision

Posted by "spectrometerHBH (via GitHub)" <gi...@apache.org>.
spectrometerHBH commented on code in PR #14242:
URL: https://github.com/apache/tvm/pull/14242#discussion_r1142495125


##########
tests/python/relax/test_transform_to_mixed_precision.py:
##########
@@ -0,0 +1,540 @@
+# 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 numpy as np
+import tvm
+from tvm import relax
+import tvm.testing
+from tvm.relax.transform import ToMixedPrecision
+from tvm.script.parser import ir as I, relax as R
+
+
+def _assert_test(input, expected):
+    mod = ToMixedPrecision()(input)
+    tvm.ir.assert_structural_equal(mod, expected)
+
+
+def test_conv2d():
+    @I.ir_module
+    class Input:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), "float32"), w: R.Tensor((4, 3, 3, 3), "float32")
+        ) -> R.Tensor(None, "float32", ndim=4):
+            with R.dataflow():
+                gv: R.Tensor((2, 4, 26, 26), "float32") = R.nn.conv2d(x, w, out_dtype="float32")
+                R.output(gv)
+            return gv
+
+    @I.ir_module
+    class Expected:
+        @R.function
+        def main(
+            x: R.Tensor((2, 3, 28, 28), dtype="float32"), w: R.Tensor((4, 3, 3, 3), dtype="float32")
+        ) -> R.Tensor((2, 4, 26, 26), dtype="float32"):
+            with R.dataflow():
+                lv: R.Tensor((2, 3, 28, 28), dtype="float16") = R.astype(x, dtype="float16")
+                lv1: R.Tensor((4, 3, 3, 3), dtype="float16") = R.astype(w, dtype="float16")
+                gv: R.Tensor((2, 4, 26, 26), dtype="float32") = R.nn.conv2d(
+                    lv,
+                    lv1,
+                    strides=[1, 1],
+                    padding=[0, 0, 0, 0],
+                    dilation=[1, 1],
+                    groups=1,
+                    data_layout="NCHW",
+                    kernel_layout="OIHW",
+                    out_layout="NCHW",
+                    out_dtype="float32",

Review Comment:
   You can configure it by passing out_dtype to ToMixedPrecision(). It's fp32 by default.



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