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/08/20 20:28:29 UTC

[GitHub] [incubator-tvm] mbrookhart opened a new pull request #6316: Dynamic Strided Slice

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


   This PR splits the dynamic-parameter version of strided slice from the constant parameter version, improves the shape funcs to pass more unit test, and supports executing the constant parameter version of the op on dynamically shaped inputs.
   
   I was able to pass all tests except on the static op with a dynamic input tensor, I can't pass tests with negative `begin` values.
   
   There's a lot of te::compute being thrown around here, but I haven't seen a clean way to simplify it yet. I would love suggestions.
   
   Thanks!
   
   cc @zhiics @kevinthesun @yongwww @lixiaoquan @electriclilies 


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/transform.py
##########
@@ -827,13 +828,17 @@ def strided_slice(data, begin, end, strides=None, slice_mode="end"):
     ret : relay.Expr
         The computed result.
     """
-    strides = strides or const([1], dtype="int32")
-    if isinstance(begin, (tuple, list)):
-        begin = const(list(begin))
-    if isinstance(end, (tuple, list)):
-        end = const(list(end))
-    if isinstance(strides, (tuple, list)):
-        strides = const(list(strides))
+    strides = strides or [1]
+    if (isinstance(begin, Expr) or isinstance(end, Expr) or isinstance(strides, Expr)):
+        if isinstance(begin, (tuple, list)):
+            begin = const(list(begin))
+        if isinstance(end, (tuple, list)):
+            end = const(list(end))
+        if isinstance(strides, (tuple, list)):
+            strides = const(list(strides))
+        begin = _make.where(begin < cast_like(const(0), begin),

Review comment:
       Can you rename this begin for clarity?

##########
File path: python/tvm/relay/op/dyn/_transform.py
##########
@@ -145,3 +146,53 @@ def one_hot_shape_func(attrs, inputs, _):
     """
     axis = len(inputs[0].shape) if attrs.axis == -1 else attrs.axis
     return [_onehot_shape_func(inputs[0].shape, inputs[3], convert(axis))]
+
+
+@script
+def _strided_slice_shape_func_input_data(data, begin, end, strides,

Review comment:
       What's the difference between `_strided_slice_shape_func_input_shape` and `_strided_slice_shape_func_input_data`?

##########
File path: tests/python/relay/test_op_level4.py
##########
@@ -343,7 +337,7 @@ def verify(dshape, begin, end, strides, output, slice_mode="end",
         text = func.astext()
         assert "begin=" in text
         assert "end=" in text
-
+        

Review comment:
       white space!!

##########
File path: src/relay/op/dyn/tensor/transform.cc
##########
@@ -430,6 +434,114 @@ RELAY_REGISTER_OP("dyn.full")
     .set_attr<FTVMCompute>("FTVMCompute", FullCompute)
     .set_attr<TOpPattern>("TOpPattern", kElemWise);
 
+bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
+                     const TypeReporter& reporter) {
+  CHECK_EQ(types.size(), 5);

Review comment:
       It would be nice to add a comment saying what each of the input types are (ie types = [type1_description, ... ret_type]

##########
File path: src/relay/op/dyn/tensor/transform.cc
##########
@@ -430,6 +434,114 @@ RELAY_REGISTER_OP("dyn.full")
     .set_attr<FTVMCompute>("FTVMCompute", FullCompute)
     .set_attr<TOpPattern>("TOpPattern", kElemWise);
 
+bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
+                     const TypeReporter& reporter) {
+  CHECK_EQ(types.size(), 5);
+  const StridedSliceAttrs* param = attrs.as<StridedSliceAttrs>();
+  if (param == nullptr) {
+    return false;
+  }
+  const auto* data = types[0].as<TensorTypeNode>();
+  if (data == nullptr) {
+    return false;
+  }
+  auto dshape = data->shape;
+  int64_t num_axis = dshape.size();
+
+  // calculate output shape
+  std::vector<IndexExpr> oshape(num_axis);
+  for (int64_t i = 0; i < num_axis; ++i) {
+    oshape[i] = Any();
+  }
+
+  reporter->Assign(types[4], TensorType(oshape, data->dtype));
+  return true;
+}
+
+inline te::Tensor DynamicStridedSlice(const te::Tensor& input, const te::Tensor& begin,
+                                      const te::Tensor& end, const te::Tensor& strides,
+                                      std::string name = "T_strided_slice_dynamic",
+                                      std::string tag = topi::kInjective) {
+  int64_t src_tensor_dim = input->shape.size();
+  Array<IndexExpr> out_shape;
+  for (int64_t i = 0; i < src_tensor_dim; ++i) {
+    out_shape.push_back(tvm::tir::Var("dim"));
+  }
+  // TODO(yongwww): move the compute into topi
+  return te::compute(
+      out_shape,
+      [&](const Array<tvm::tir::Var>& indices) {
+        Array<IndexExpr> real_indices;
+        for (int32_t i = 0; i < src_tensor_dim; ++i) {
+          real_indices.push_back(indices[i] * strides(i) + begin(i));
+        }
+        return input(real_indices);
+      },
+      name, tag);
+}
+
+Array<te::Tensor> StridedSliceCompute(const Attrs& attrs, const Array<te::Tensor>& inputs,
+                                      const Type& out_type) {
+  te::Tensor data = inputs[0];
+  te::Tensor begin = inputs[1];
+  te::Tensor end = inputs[2];
+  te::Tensor strides = inputs[3];
+  // Dynamic computation
+  int64_t attr_size = data->shape.size();

Review comment:
       Does this stand for attribute size? If so, the name seems a bit inaccurate

##########
File path: src/relay/op/dyn/tensor/transform.cc
##########
@@ -430,6 +434,114 @@ RELAY_REGISTER_OP("dyn.full")
     .set_attr<FTVMCompute>("FTVMCompute", FullCompute)
     .set_attr<TOpPattern>("TOpPattern", kElemWise);
 
+bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
+                     const TypeReporter& reporter) {
+  CHECK_EQ(types.size(), 5);
+  const StridedSliceAttrs* param = attrs.as<StridedSliceAttrs>();
+  if (param == nullptr) {
+    return false;
+  }
+  const auto* data = types[0].as<TensorTypeNode>();
+  if (data == nullptr) {
+    return false;
+  }
+  auto dshape = data->shape;
+  int64_t num_axis = dshape.size();
+
+  // calculate output shape
+  std::vector<IndexExpr> oshape(num_axis);
+  for (int64_t i = 0; i < num_axis; ++i) {
+    oshape[i] = Any();
+  }
+
+  reporter->Assign(types[4], TensorType(oshape, data->dtype));
+  return true;
+}
+
+inline te::Tensor DynamicStridedSlice(const te::Tensor& input, const te::Tensor& begin,
+                                      const te::Tensor& end, const te::Tensor& strides,
+                                      std::string name = "T_strided_slice_dynamic",
+                                      std::string tag = topi::kInjective) {
+  int64_t src_tensor_dim = input->shape.size();
+  Array<IndexExpr> out_shape;
+  for (int64_t i = 0; i < src_tensor_dim; ++i) {
+    out_shape.push_back(tvm::tir::Var("dim"));
+  }
+  // TODO(yongwww): move the compute into topi
+  return te::compute(
+      out_shape,
+      [&](const Array<tvm::tir::Var>& indices) {
+        Array<IndexExpr> real_indices;
+        for (int32_t i = 0; i < src_tensor_dim; ++i) {
+          real_indices.push_back(indices[i] * strides(i) + begin(i));
+        }
+        return input(real_indices);
+      },
+      name, tag);
+}
+
+Array<te::Tensor> StridedSliceCompute(const Attrs& attrs, const Array<te::Tensor>& inputs,
+                                      const Type& out_type) {
+  te::Tensor data = inputs[0];
+  te::Tensor begin = inputs[1];
+  te::Tensor end = inputs[2];
+  te::Tensor strides = inputs[3];
+  // Dynamic computation
+  int64_t attr_size = data->shape.size();
+  CHECK(begin->shape[0].as<IntImmNode>()->value == attr_size &&
+        end->shape[0].as<IntImmNode>()->value == attr_size &&
+        strides->shape[0].as<IntImmNode>()->value == attr_size)
+      << "begin, end, and strides are required to have the same length"
+      << " if they are non-constant.";

Review comment:
       The wording of this error is a bit confusing; "begin, end, and strides are required to have the same length or must all be constants" might be better

##########
File path: src/relay/op/tensor/transform.cc
##########
@@ -2069,12 +2070,9 @@ bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attr
       oshape[i] = tir::make_const(dshape[i].dtype(), (slice_range + step - 1) / step);
     }
   } else {
-    for (int64_t i = 0; i < num_axis; ++i) {
-      oshape[i] = Any();
-    }
+    CHECK(false) << "strided_slice recieved invalid params";

Review comment:
       You could state in this error that strided_slice received an incorrect beginning, end, or strides tensor. 

##########
File path: src/relay/transforms/dynamic_to_static.cc
##########
@@ -139,6 +139,24 @@ class DynamicToStaticMutator : public MixedModeMutator {
            }
            return Expr(nullptr);
          }},
+        {Op::Get("dyn.strided_slice"),
+         [](const CallNode* call_node) {
+           if (const ConstantNode* begin = call_node->args[1].as<ConstantNode>()) {
+             if (const ConstantNode* end = call_node->args[2].as<ConstantNode>()) {

Review comment:
       It would be cleaner to pull these definitions out of the if statements, and then check whether they are null or not in one if statement, though potentially slower




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   The failing unit test is a symptom of the bug fixed here: https://github.com/apache/incubator-tvm/pull/6321
   
   I'll rebase this PR once that's merged.


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: src/relay/op/dyn/tensor/transform.cc
##########
@@ -430,6 +434,114 @@ RELAY_REGISTER_OP("dyn.full")
     .set_attr<FTVMCompute>("FTVMCompute", FullCompute)
     .set_attr<TOpPattern>("TOpPattern", kElemWise);
 
+bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attrs,
+                     const TypeReporter& reporter) {
+  CHECK_EQ(types.size(), 5);

Review comment:
       Done.




----------------------------------------------------------------
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] masahi commented on pull request #6316: Dynamic Strided Slice

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


   @mbrookhart #6314 introduced another strided slice usage, you might need to update that too.


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   I don't think so, it's already targeting the dynamic op
   
   ```
           # squeeze the two outputs of nms for strided_slice
           size = get_relay_op("squeeze")(nms_ret[1], axis=[1])
           data_slice = get_relay_op("squeeze")(nms_ret[0], axis=[0])
   
           # strided slice to get the dynamic result
           return get_relay_op("strided_slice")(data_slice, begin=_expr.const([0]),
                                                end=size, slice_mode="size")
   ```


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/transform.py
##########
@@ -827,13 +828,17 @@ def strided_slice(data, begin, end, strides=None, slice_mode="end"):
     ret : relay.Expr
         The computed result.
     """
-    strides = strides or const([1], dtype="int32")
-    if isinstance(begin, (tuple, list)):
-        begin = const(list(begin))
-    if isinstance(end, (tuple, list)):
-        end = const(list(end))
-    if isinstance(strides, (tuple, list)):
-        strides = const(list(strides))
+    strides = strides or [1]
+    if (isinstance(begin, Expr) or isinstance(end, Expr) or isinstance(strides, Expr)):
+        if isinstance(begin, (tuple, list)):
+            begin = const(list(begin))
+        if isinstance(end, (tuple, list)):
+            end = const(list(end))
+        if isinstance(strides, (tuple, list)):
+            strides = const(list(strides))
+        normalized_begin = _make.where(begin < cast_like(const(0), begin),

Review comment:
       Hmm, yeah, seems a little odd to produce a subgraph as part of the constructor of an op, which is why I put it here. That being said, this makes the op less useful from other frontends, so...
   
   Any other votes?




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   @zhiics Could you take another 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] mbrookhart commented on pull request #6316: Dynamic Strided Slice

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


   The unit test error appears to be a bug with the cython backend, we're getting int64t overflow if we use cython, but not with ctypes.


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   @mbrookhart no problem
   
   BTW, I would suggest moving the compute to topi and make dynamic version there in the next PR. Do we need to change the topi strided_slice definition to let it use dynamic/static version?


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/transform.py
##########
@@ -827,13 +828,17 @@ def strided_slice(data, begin, end, strides=None, slice_mode="end"):
     ret : relay.Expr
         The computed result.
     """
-    strides = strides or const([1], dtype="int32")
-    if isinstance(begin, (tuple, list)):
-        begin = const(list(begin))
-    if isinstance(end, (tuple, list)):
-        end = const(list(end))
-    if isinstance(strides, (tuple, list)):
-        strides = const(list(strides))
+    strides = strides or [1]
+    if (isinstance(begin, Expr) or isinstance(end, Expr) or isinstance(strides, Expr)):
+        if isinstance(begin, (tuple, list)):
+            begin = const(list(begin))
+        if isinstance(end, (tuple, list)):
+            end = const(list(end))
+        if isinstance(strides, (tuple, list)):
+            strides = const(list(strides))
+        normalized_begin = _make.where(begin < cast_like(const(0), begin),

Review comment:
       We'd have to reproduce this on any frontend that creates ops.




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: src/relay/op/tensor/transform.cc
##########
@@ -2069,12 +2070,11 @@ bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attr
       oshape[i] = tir::make_const(dshape[i].dtype(), (slice_range + step - 1) / step);
     }
   } else {
-    for (int64_t i = 0; i < num_axis; ++i) {
-      oshape[i] = Any();
-    }
+    CHECK(param->begin) << "strided_slice recieved invalid begin";

Review comment:
       If this check fails it will be a nullptr. Should I print the null?




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   @yongwww @electriclilies Can you take another 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 pull request #6316: Dynamic Strided Slice

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


   @masahi @electriclilies could you take another look and approve/comment?


----------------------------------------------------------------
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] yongwww commented on a change in pull request #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/transform.py
##########
@@ -827,13 +828,17 @@ def strided_slice(data, begin, end, strides=None, slice_mode="end"):
     ret : relay.Expr
         The computed result.
     """
-    strides = strides or const([1], dtype="int32")
-    if isinstance(begin, (tuple, list)):
-        begin = const(list(begin))
-    if isinstance(end, (tuple, list)):
-        end = const(list(end))
-    if isinstance(strides, (tuple, list)):
-        strides = const(list(strides))
+    strides = strides or [1]
+    if (isinstance(begin, Expr) or isinstance(end, Expr) or isinstance(strides, Expr)):
+        if isinstance(begin, (tuple, list)):
+            begin = const(list(begin))
+        if isinstance(end, (tuple, list)):
+            end = const(list(end))
+        if isinstance(strides, (tuple, list)):
+            strides = const(list(strides))
+        normalized_begin = _make.where(begin < cast_like(const(0), begin),

Review comment:
       we could consider moving the normalization step into strided_slice

##########
File path: python/tvm/relay/op/_transform.py
##########
@@ -165,6 +138,8 @@ def _strided_slice_shape_func_input_shape(data_shape, begin, end, strides, slice
             cstride = int64(strides[i])
         if len(begin) > i:
             cbegin = int64(begin[i])
+            if cbegin < 0:
+                cbegin += int64(data_shape[i])

Review comment:
       how about cbegin is still less than 0 after adding data_shape[i]?  probably raise error for invalid data input.

##########
File path: src/relay/op/tensor/transform.cc
##########
@@ -2069,12 +2070,11 @@ bool StridedSliceRel(const Array<Type>& types, int num_inputs, const Attrs& attr
       oshape[i] = tir::make_const(dshape[i].dtype(), (slice_range + step - 1) / step);
     }
   } else {
-    for (int64_t i = 0; i < num_axis; ++i) {
-      oshape[i] = Any();
-    }
+    CHECK(param->begin) << "strided_slice recieved invalid begin";

Review comment:
       print the received begin value in the message? 




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/frontend/mxnet.py
##########
@@ -545,8 +545,8 @@ def _mx_slice_axis(inputs, attrs):
             begin.append(ax_beg)
             end.append(ax_end)
     return _op.strided_slice(inputs[0],
-                             _expr.const(begin, dtype="int32"),
-                             _expr.const(end, dtype="int32"))
+                             begin,
+                             end)

Review comment:
       I think this could be one-liner




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/transform.py
##########
@@ -827,13 +828,17 @@ def strided_slice(data, begin, end, strides=None, slice_mode="end"):
     ret : relay.Expr
         The computed result.
     """
-    strides = strides or const([1], dtype="int32")
-    if isinstance(begin, (tuple, list)):
-        begin = const(list(begin))
-    if isinstance(end, (tuple, list)):
-        end = const(list(end))
-    if isinstance(strides, (tuple, list)):
-        strides = const(list(strides))
+    strides = strides or [1]
+    if (isinstance(begin, Expr) or isinstance(end, Expr) or isinstance(strides, Expr)):
+        if isinstance(begin, (tuple, list)):
+            begin = const(list(begin))
+        if isinstance(end, (tuple, list)):
+            end = const(list(end))
+        if isinstance(strides, (tuple, list)):
+            strides = const(list(strides))
+        normalized_begin = _make.where(begin < cast_like(const(0), begin),

Review comment:
       How does this make it less useful?




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   > @mbrookhart #6337 has been merged. You can try to enable the GPU tests now
   
   I think I'd like to do it as a separate PR after we get this merged, I don't want to conflate the strided slice op with the others too much, but I'm excited to start on this :) I'll split a branch and start playing with that.


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

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



[GitHub] [incubator-tvm] zhiics commented on pull request #6316: Dynamic Strided Slice

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


   Thanks @mbrookhart @yongwww @masahi @electriclilies 


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/dyn/_transform.py
##########
@@ -145,3 +146,53 @@ def one_hot_shape_func(attrs, inputs, _):
     """
     axis = len(inputs[0].shape) if attrs.axis == -1 else attrs.axis
     return [_onehot_shape_func(inputs[0].shape, inputs[3], convert(axis))]
+
+
+@script
+def _strided_slice_shape_func_input_data(data, begin, end, strides,

Review comment:
       For the static op, we only require the input shapes, since the tributes are static, thus the input_shape variant. For the dynamic op, we need the input data, since the attributes are coming in as values. Thus the input_data variant. Why the VM is set up to separate those instead of just sending in data all the time, I'm not sure. Optimization?




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   @mbrookhart please rebase


----------------------------------------------------------------
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] yongwww commented on pull request #6316: Dynamic Strided Slice

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


   cc @kevinthesun  @zhiics 


----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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



##########
File path: python/tvm/relay/op/_transform.py
##########
@@ -165,6 +138,8 @@ def _strided_slice_shape_func_input_shape(data_shape, begin, end, strides, slice
             cstride = int64(strides[i])
         if len(begin) > i:
             cbegin = int64(begin[i])
+            if cbegin < 0:
+                cbegin += int64(data_shape[i])

Review comment:
       I attempted to do this with an assert in the hybrid script, but something seems a little off in the compiler, even after these lines it was still checking against the original value.




----------------------------------------------------------------
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 #6316: Dynamic Strided Slice

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


   


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