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

[GitHub] [incubator-tvm] electriclilies opened a new pull request #6007: Dynamic broadcast_to

electriclilies opened a new pull request #6007:
URL: https://github.com/apache/incubator-tvm/pull/6007


   This PR adds a dynamic version of the relay broadcast_to OP, and refactors the static broadcast_to OP to be purely static (not dynamic and static combined: undoes some of the refactor in https://github.com/apache/incubator-tvm/pull/5459/files). Also adds broadcast_to to the dynamic to static conversion pass. 


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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


   @electriclilies Could you modify the code according to the changes in #6047


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

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



[GitHub] [incubator-tvm] zhiics merged pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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


   


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

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



[GitHub] [incubator-tvm] mbrookhart commented on a change in pull request #6007: [RELAY][DYN] Dynamic broadcast_to

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



##########
File path: src/relay/transforms/pattern_util.h
##########
@@ -330,6 +331,29 @@ static inline Constant CheckConstantShape(const Array<IndexExpr>& shape) {
   return Constant(shape_array);
 }
 
+/*!
+ * \brief Check whether a shape is static and create corresponding Array<Integer>. Will replace
+ * CheckConstantShape after dynamic refactorization is complete
+ *
+ * \param shape The Array of the shape values.
+ * \return A Constant.
+ */
+static inline Array<Integer> CheckConstantShapeArrayInteger(const Array<IndexExpr>& shape) {
+  Array<Integer> constShape;
+  // auto shape_array =
+  //     runtime::NDArray::Empty({int64_t(shape.size())}, DataType::Int(64), {kDLCPU, 0});
+  // auto* shape_data = static_cast<int64_t*>(shape_array->data);

Review comment:
       Remove commented code

##########
File path: src/relay/transforms/pattern_util.h
##########
@@ -330,6 +331,29 @@ static inline Constant CheckConstantShape(const Array<IndexExpr>& shape) {
   return Constant(shape_array);
 }
 
+/*!
+ * \brief Check whether a shape is static and create corresponding Array<Integer>. Will replace
+ * CheckConstantShape after dynamic refactorization is complete
+ *
+ * \param shape The Array of the shape values.
+ * \return A Constant.
+ */
+static inline Array<Integer> CheckConstantShapeArrayInteger(const Array<IndexExpr>& shape) {
+  Array<Integer> constShape;
+  // auto shape_array =
+  //     runtime::NDArray::Empty({int64_t(shape.size())}, DataType::Int(64), {kDLCPU, 0});
+  // auto* shape_data = static_cast<int64_t*>(shape_array->data);
+  for (size_t i = 0; i < shape.size(); ++i) {
+    const auto& dim_val = shape[i].as<IntImmNode>();
+    CHECK(dim_val) << "Do not support symbolic shape for "
+                      "Array format. Pass shape as Expr instead.";
+
+    // shape_data[i] = dim_val->value;

Review comment:
       Remove Commented Code

##########
File path: src/relay/transforms/dynamic_to_static.cc
##########
@@ -56,6 +58,30 @@ class DynamicToStaticMutator : public MixedModeMutator {
         return Call(op, {call_node->args[0]}, Attrs(attrs), {});
       }
     }
+    if (call_node->op == dyn_broadcast_to_op_) {
+      if (const ConstantNode* shape = call_node->args[1].as<ConstantNode>()) {
+        auto attrs = make_object<InitOpAttrs>();
+        CHECK_EQ(shape->data->ndim, 1);
+
+        // put shape in attrs
+        attrs->shape = ToVector(shape->data);
+        static const Op& broadcast_to = Op::Get("broadcast_to");
+        // pass in one arg to static broadcast to
+        return Call(broadcast_to, {call_node->args[0]}, Attrs(attrs), {});
+      }
+    }
+    if (call_node->op == dyn_broadcast_to_op_) {
+      if (const ConstantNode* shape = call_node->args[1].as<ConstantNode>()) {
+        auto attrs = make_object<InitOpAttrs>();
+        CHECK_EQ(shape->data->ndim, 1);
+
+        // put shape in attrs
+        attrs->shape = ToVector(shape->data);
+        static const Op& broadcast_to = Op::Get("broadcast_to");
+        // pass in one arg to static broadcast to
+        return Call(broadcast_to, {call_node->args[0]}, Attrs(attrs), {});
+      }
+    }

Review comment:
       You have two copies of this code block, remove one.

##########
File path: src/relay/op/dyn/tensor/transform.cc
##########
@@ -36,6 +37,8 @@ namespace relay {
 namespace dyn {
 
 /* relay.dyn.reshape */
+// TVM_REGISTER_NODE_TYPE(ReshapeAttrs);
+

Review comment:
       Remove

##########
File path: tests/python/relay/test_op_level10.py
##########
@@ -455,13 +457,13 @@ def _verify(indices_shape, depth, on_value, off_value, axis, dtype):
         func = relay.Function([indices], out)
         indices_np = np.random.randint(0, depth, size=indices_shape).astype("int32")
         out_np = topi.testing.one_hot(indices_np, on_value, off_value, depth, axis, dtype)
-
+        
         for target, ctx in ctx_list():
             for kind in ["graph", "debug"]:
                 intrp = relay.create_executor(kind, ctx=ctx, target=target)
                 out_relay = intrp.evaluate(func)(indices_np)
                 tvm.testing.assert_allclose(out_relay.asnumpy(), out_np)
-
+               

Review comment:
       delete extra whitespace

##########
File path: tests/python/relay/test_op_level10.py
##########
@@ -455,13 +457,13 @@ def _verify(indices_shape, depth, on_value, off_value, axis, dtype):
         func = relay.Function([indices], out)
         indices_np = np.random.randint(0, depth, size=indices_shape).astype("int32")
         out_np = topi.testing.one_hot(indices_np, on_value, off_value, depth, axis, dtype)
-
+        

Review comment:
       delete extra whitespace

##########
File path: tests/python/relay/dyn/test_dynamic_op_level10.py
##########
@@ -0,0 +1,57 @@
+# 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.
+""" 
+Support level10 operator test cases.
+
+"""
+
+
+import numpy as np
+import tvm
+from tvm import te
+import topi.testing
+from tvm import relay
+from tvm.relay import transform
+from tvm.relay.testing import ctx_list, run_infer_type
+import topi
+import topi.testing
+import random

Review comment:
       You don't use some of these imports?

##########
File path: python/tvm/relay/op/dyn/transform.py
##########
@@ -0,0 +1,93 @@
+# 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.
+
+# pylint: disable=import-outside-toplevel
+"""Dynamic Transform operators."""
+
+from . import _make
+
+
+def reshape(data, newshape):
+    """Reshape the input array based on the values of the newshape tensor.
+
+    To give user more convenience in without doing manual shape inference,
+    some dimensions of the shape can take special values from the set {0, -1, -3}.
+    The significance of each is explained below:
+
+    ``0`` copy this dimension from the input to the output shape.
+
+        .. code-block:: python
+
+            data.shape = (2,3,4), newshape = (4,0,2), result.shape = (4,3,2)
+            data.shape = (2,3,4), newshape = (2,0,0), result.shape = (2,3,4)
+
+    ``-1`` infers the dimension of the output shape by using the remainder of
+    the input dimensions keeping the size of the new array same as that of the input array.
+    At most one dimension of shape can be -1.
+
+        .. code-block:: python
+
+            data.shape = (2,3,4), newshape = (6,1,-1), result.shape = (6,1,4)
+            data.shape = (2,3,4), newshape = (3,-1,8), result.shape = (3,1,8)
+            data.shape = (2,3,4), newshape = (-1,), result.shape = (24,)
+
+    ``-3`` use the product of two consecutive dimensions of the input shape
+    as the output dimension.
+
+        .. code-block:: python
+
+            data.shape = (2,3,4), newshape = (-3,4), result.shape = (6,4)
+            data.shape = (2,3,4,5), newshape = (-3,-3), result.shape = (6,20)
+            data.shape = (2,3,4), newshape = (0,-3), result.shape = (2,12)
+
+    Special values -2 and -4 from the standard reshape op would introduce dynamic rank
+    in this op. Thus, they are not permitted.
+
+    Parameters
+    ----------
+    data : relay.Expr
+        The input data to the operator.
+
+    newshape : relay.Expr
+        The new shape. Should be compatible with the original shape.
+
+    Returns
+    -------
+    result : relay.Expr
+        The reshaped result.
+    """
+    return _make.reshape(data, newshape)
+
+def broadcast_to(data, shape):
+
+    """Return a scalar value array with the same type, broadcast to
+    the provided shape.
+
+    Parameters
+    ----------
+    data : relay.Expr
+        The input tensor.
+
+    shape : a relay.Expr, cannot be a tuple of consts
+        Provide the shape to broadcast to.
+
+    Returns
+    -------
+    result : relay.Expr
+        The resulting tensor.
+    """
+    return _make.broadcast_to(data, shape)

Review comment:
       Remove this file, since you're providing access to the op via the standard interface in python/tvm/relay/op/transform.py




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

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



[GitHub] [incubator-tvm] mbrookhart commented on pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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


   @zhiics could you take a look?


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

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



[GitHub] [incubator-tvm] zhiics commented on a change in pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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



##########
File path: python/tvm/relay/op/_tensor.py
##########
@@ -92,7 +92,7 @@
 # zeros
 @register_compute("zeros")
 def zeros_compute(attrs, inputs, output_type):
-    assert len(inputs) == 1
+    assert len(inputs) == 0

Review comment:
       assert not inputs?

##########
File path: python/tvm/relay/op/_tensor.py
##########
@@ -109,7 +109,7 @@ def zeros_like_compute(attrs, inputs, output_type):
 # ones
 @register_compute("ones")
 def ones_compute(attrs, inputs, output_type):
-    assert len(inputs) == 1
+    assert len(inputs) == 0

Review comment:
       assert not inputs?




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

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



[GitHub] [incubator-tvm] electriclilies commented on a change in pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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



##########
File path: python/tvm/relay/op/_tensor.py
##########
@@ -109,7 +109,7 @@ def zeros_like_compute(attrs, inputs, output_type):
 # ones
 @register_compute("ones")
 def ones_compute(attrs, inputs, output_type):
-    assert len(inputs) == 1
+    assert len(inputs) == 0

Review comment:
       Will fix
   




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

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



[GitHub] [incubator-tvm] electriclilies commented on a change in pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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



##########
File path: python/tvm/relay/op/_tensor.py
##########
@@ -92,7 +92,7 @@
 # zeros
 @register_compute("zeros")
 def zeros_compute(attrs, inputs, output_type):
-    assert len(inputs) == 1
+    assert len(inputs) == 0

Review comment:
       Thanks, I will fix this




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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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


   Thanks @electriclilies @mbrookhart 


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

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



[GitHub] [incubator-tvm] electriclilies commented on pull request #6007: [RELAY][DYN] Dynamic broadcast_to, zeros, ones

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


   @zhiics Rebased and nitpicks fixed!


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

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