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

[GitHub] [incubator-tvm] maheshambule opened a new pull request #5082: Unravel index op

maheshambule opened a new pull request #5082: Unravel index op
URL: https://github.com/apache/incubator-tvm/pull/5082
 
 
   Thanks for contributing to TVM!   Please refer to guideline https://docs.tvm.ai/contribute/ for useful information and tips. After the pull request is submitted, please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @ them in the pull request thread.
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r395876221
 
 

 ##########
 File path: topi/include/topi/transform.h
 ##########
 @@ -232,6 +232,52 @@ inline Tensor reshape(const Tensor& x,
   }
 }
 
+/*!
+ * \brief Converts a flat index or array of flat indices into a tuple of coordinate arrays
+ *
+ * \param x The input tensor having indices.
+ * \param shape The shape tensor
+ * \param name The name of the operation
+ * \param tag The tag to mark the operation
+ *
+ * \return A Tensor of coordinate arrays.
+ */
+
+inline Tensor unravel_index(const Tensor& x, const Tensor& shape, std::string name = "T_unravel",
+                            std::string tag = kInjective) {
+  auto x_shape = x->shape;
+  auto shape_shape = shape->shape;
+
+  Array<PrimExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (x_shape.size() != 0) {
+    oshape.push_back(x_shape[0]);
+  }
+
+  return compute(oshape,
+                 [&](const Array<Var>& indices) {
+                   auto i = indices[0];
+                   std::vector<PrimExpr> indices_divs;
+                   PrimExpr ret = 0;
+                   PrimExpr cur_val = 0;
+                   PrimExpr index_val = 0;
+
+                   if (x_shape.size() != 0) {
+                     index_val = x[indices[1]];
+                   } else {
+                     index_val = x();
+                   }
+                   indices_divs.push_back(index_val);
+                   for (int v = GetConstInt(shape_shape[0]) - 1; v >= 0; --v) {
+                     ret = tvm::if_then_else(i == v, indexmod(indices_divs.back(), shape[v]), ret);
+                     cur_val = indexdiv(indices_divs.back(), shape[v]);
+                     indices_divs.push_back(cur_val);
+                   }
+                   return ret;
+                 },
+                 name, tag);
+}
+
 
 Review comment:
   Changed. Actually I used IDE's code formatter.  I will follow this format going forward.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet]Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet]Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r393641775
 
 

 ##########
 File path: python/tvm/relay/frontend/tensorflow.py
 ##########
 @@ -2509,9 +2515,7 @@ def _parse_param(self, key, value, name, shape):
 
             array_ndim = len(np_array.shape)
             if array_ndim == 0:
-                new_array = np.empty([1], dtype=np_array.dtype)
 
 Review comment:
   Removed this because we want to pass the scalar as scalar only and not as a tensor of rank 1.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r395434226
 
 

 ##########
 File path: src/relay/op/tensor/transform.cc
 ##########
 @@ -2703,5 +2701,67 @@ RELAY_REGISTER_OP("one_hot")
 .set_attr<FTVMCompute>("FTVMCompute", OneHotCompute)
 .set_attr<TOpPattern>("TOpPattern", kOutEWiseFusable);
 
+/* relay.unravel_index */
+bool UnRavelIndexRel(const Array<Type>& types,
+                     int num_inputs,
+                     const Attrs& attrs,
+                     const TypeReporter& reporter) {
+  CHECK_EQ(types.size(), 3);
+
+  const auto* indices = types[0].as<TensorTypeNode>();
+  if (indices == nullptr) {
+    CHECK(types[0].as<IncompleteTypeNode>())
+        << "unravel_index: expect input type to be TensorType but get " << types[0];
+    return false;
+  }
+  CHECK(indices->dtype.is_int()) << "indices of unravel_index must be tensor of integer";
+
+  const auto* shape = types[1].as<TensorTypeNode>();
+  if (shape == nullptr) {
+    CHECK(types[1].as<IncompleteTypeNode>())
+        << "unravel_index: expect input type to be TensorType but get " << types[1];
+    return false;
+  }
+  CHECK(indices->dtype.is_int()) << "shape of unravel_index must be tensor of integer";
+
+  Array<IndexExpr> indices_shape;
+  Array<IndexExpr> shape_shape;
+  indices_shape = indices->shape;
+  shape_shape = shape->shape;
+
+  Array<IndexExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (indices_shape.size() != 0) {
+    oshape.push_back(indices_shape[0]);
+  }
+  reporter->Assign(types[2], TensorType(oshape, indices->dtype));
+  return true;
+}
+
+Array<te::Tensor> UnRavelIndexCompute(const Attrs& attrs,
+                                      const Array<te::Tensor>& inputs,
+                                      const Type& out_type) {
+  return Array<te::Tensor>{topi::unravel_index(inputs[0], inputs[1])};
+}
+
+Expr MakeUnRavelIndex(Expr data, Expr shape) {
+  static const Op& op = Op::Get("unravel_index");
+  return CallNode::make(op, {data, shape}, Attrs(), {});
+}
+
+TVM_REGISTER_GLOBAL("relay.op._make.unravel_index").set_body_typed(MakeUnRavelIndex);
+
+RELAY_REGISTER_OP("unravel_index")
+    .describe(
+        R"code(Converts a flat index or array of flat indices into a tuple of coordinate arrays.
+Example::
+    -   unravel_index([22, 41, 37], (7, 6)) = [[3, 6, 6], [4, 5, 1]]
+)code" TVM_ADD_FILELINE)
+    .set_num_inputs(2)
+    .set_support_level(3)
+    .add_type_rel("UnRavelIndexRel", UnRavelIndexRel)
+    .set_attr<FTVMCompute>("FTVMCompute", UnRavelIndexCompute)
+    .set_attr<TOpPattern>("TOpPattern", kInjective);
+
 
 Review comment:
   The indentation seems to be messed up. Please make sure it is the same as other op.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on issue #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on issue #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#issuecomment-602539164
 
 
   Thanks @jwfromm, @masahi for review and comments.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi merged pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
masahi merged pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082
 
 
   

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jwfromm commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
jwfromm commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r393924803
 
 

 ##########
 File path: include/tvm/relay/attrs/transform.h
 ##########
 @@ -321,6 +321,12 @@ struct ArgWhereAttrs : public tvm::AttrsNode<ArgWhereAttrs> {
   }
 };  // struct ArgWhereAttrs
 
+/*! \brief Attributes used in unravel_index operators */
+struct UnRavelIndexAttrs : public tvm::AttrsNode<UnRavelIndexAttrs> {
+  TVM_DECLARE_ATTRS(UnRavelIndexAttrs, "relay.attrs.UnRavelIndexAttrs") {
 
 Review comment:
   I don't think there's any need to define an attribute type for an operator without attributes. Although `argwhere` seems to do the same thing you have, other operators without attributes just don't use one (see `nn.batch_flatten` as one example). I'd argue we should try to avoid defining unnecessary attrs to prevent bloat.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jwfromm commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
jwfromm commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r393921461
 
 

 ##########
 File path: topi/include/topi/transform.h
 ##########
 @@ -232,6 +232,52 @@ inline Tensor reshape(const Tensor& x,
   }
 }
 
+/*!
+ * \brief Converts a flat index or array of flat indices into a tuple of coordinate arrays
+ *
+ * \param x The input tensor having indices.
+ * \param shape The shape tensor
+ * \param name The name of the operation
+ * \param tag The tag to mark the operation
+ *
+ * \return A Tensor of coordinate arrays.
+ */
+
+inline Tensor unravel_index(const Tensor& x, const Tensor& shape, std::string name = "T_unravel",
+                            std::string tag = kInjective) {
+  auto x_shape = x->shape;
+  auto shape_shape = shape->shape;
+
+  Array<PrimExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (x_shape.size() != 0) {
+    oshape.push_back(x_shape[0]);
+  }
+
+  return compute(oshape,
+                 [&](const Array<Var>& indices) {
+                   auto i = indices[0];
+                   std::vector<PrimExpr> indices_divs;
+                   PrimExpr ret = 0;
+                   PrimExpr cur_val = 0;
+                   PrimExpr index_val = 0;
+
+                   if (x_shape.size() != 0) {
+                     index_val = x[indices[1]];
+                   } else {
+                     index_val = x();
+                   }
+                   indices_divs.push_back(index_val);
+                   for (int v = GetConstInt(shape_shape[0]) - 1; v >= 0; --v) {
+                     ret = tvm::if_then_else(i == v, indexmod(indices_divs.back(), shape[v]), ret);
+                     cur_val = indexdiv(indices_divs.back(), shape[v]);
+                     indices_divs.push_back(cur_val);
 
 Review comment:
   Is there a reason that `UnravelIndex` from `topi/include/topi/detail/ravel_unravel.h` isn't used here? 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r394577194
 
 

 ##########
 File path: topi/include/topi/transform.h
 ##########
 @@ -232,6 +232,52 @@ inline Tensor reshape(const Tensor& x,
   }
 }
 
+/*!
+ * \brief Converts a flat index or array of flat indices into a tuple of coordinate arrays
+ *
+ * \param x The input tensor having indices.
+ * \param shape The shape tensor
+ * \param name The name of the operation
+ * \param tag The tag to mark the operation
+ *
+ * \return A Tensor of coordinate arrays.
+ */
+
+inline Tensor unravel_index(const Tensor& x, const Tensor& shape, std::string name = "T_unravel",
+                            std::string tag = kInjective) {
+  auto x_shape = x->shape;
+  auto shape_shape = shape->shape;
+
+  Array<PrimExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (x_shape.size() != 0) {
+    oshape.push_back(x_shape[0]);
+  }
+
+  return compute(oshape,
+                 [&](const Array<Var>& indices) {
+                   auto i = indices[0];
+                   std::vector<PrimExpr> indices_divs;
+                   PrimExpr ret = 0;
+                   PrimExpr cur_val = 0;
+                   PrimExpr index_val = 0;
+
+                   if (x_shape.size() != 0) {
+                     index_val = x[indices[1]];
+                   } else {
+                     index_val = x();
+                   }
+                   indices_divs.push_back(index_val);
+                   for (int v = GetConstInt(shape_shape[0]) - 1; v >= 0; --v) {
+                     ret = tvm::if_then_else(i == v, indexmod(indices_divs.back(), shape[v]), ret);
+                     cur_val = indexdiv(indices_divs.back(), shape[v]);
+                     indices_divs.push_back(cur_val);
 
 Review comment:
   The function in this file returns all the coordinates for a given index. In compute definition we just want a coordinate for the current compute index and not for all of them. I was facing issue while extracting the current coordinate because compute index which is a Var can not be directly used to extract Expr from an array of Exprs. I had to use if_then_else construct for that. Please let me know if I am missing something here and if there is an easier way to achieve this.  I could have modified the existing function to meet my purposes for example pass in the coordinate index I want to extract and return just that coordinate. Please let me know if I should implement 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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r394556662
 
 

 ##########
 File path: include/tvm/relay/attrs/transform.h
 ##########
 @@ -321,6 +321,12 @@ struct ArgWhereAttrs : public tvm::AttrsNode<ArgWhereAttrs> {
   }
 };  // struct ArgWhereAttrs
 
+/*! \brief Attributes used in unravel_index operators */
+struct UnRavelIndexAttrs : public tvm::AttrsNode<UnRavelIndexAttrs> {
+  TVM_DECLARE_ATTRS(UnRavelIndexAttrs, "relay.attrs.UnRavelIndexAttrs") {
 
 Review comment:
   Ok. Thanks. This is good to know. I have removed the attrs for both unravel_index and argwhere.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] jwfromm commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
jwfromm commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r395263725
 
 

 ##########
 File path: topi/include/topi/transform.h
 ##########
 @@ -232,6 +232,52 @@ inline Tensor reshape(const Tensor& x,
   }
 }
 
+/*!
+ * \brief Converts a flat index or array of flat indices into a tuple of coordinate arrays
+ *
+ * \param x The input tensor having indices.
+ * \param shape The shape tensor
+ * \param name The name of the operation
+ * \param tag The tag to mark the operation
+ *
+ * \return A Tensor of coordinate arrays.
+ */
+
+inline Tensor unravel_index(const Tensor& x, const Tensor& shape, std::string name = "T_unravel",
+                            std::string tag = kInjective) {
+  auto x_shape = x->shape;
+  auto shape_shape = shape->shape;
+
+  Array<PrimExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (x_shape.size() != 0) {
+    oshape.push_back(x_shape[0]);
+  }
+
+  return compute(oshape,
+                 [&](const Array<Var>& indices) {
+                   auto i = indices[0];
+                   std::vector<PrimExpr> indices_divs;
+                   PrimExpr ret = 0;
+                   PrimExpr cur_val = 0;
+                   PrimExpr index_val = 0;
+
+                   if (x_shape.size() != 0) {
+                     index_val = x[indices[1]];
+                   } else {
+                     index_val = x();
+                   }
+                   indices_divs.push_back(index_val);
+                   for (int v = GetConstInt(shape_shape[0]) - 1; v >= 0; --v) {
+                     ret = tvm::if_then_else(i == v, indexmod(indices_divs.back(), shape[v]), ret);
+                     cur_val = indexdiv(indices_divs.back(), shape[v]);
+                     indices_divs.push_back(cur_val);
 
 Review comment:
   Ah ok that makes sense. This implementation is good then, no need to change it.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r395875063
 
 

 ##########
 File path: src/relay/op/tensor/transform.cc
 ##########
 @@ -2703,5 +2701,67 @@ RELAY_REGISTER_OP("one_hot")
 .set_attr<FTVMCompute>("FTVMCompute", OneHotCompute)
 .set_attr<TOpPattern>("TOpPattern", kOutEWiseFusable);
 
+/* relay.unravel_index */
+bool UnRavelIndexRel(const Array<Type>& types,
+                     int num_inputs,
+                     const Attrs& attrs,
+                     const TypeReporter& reporter) {
+  CHECK_EQ(types.size(), 3);
+
+  const auto* indices = types[0].as<TensorTypeNode>();
+  if (indices == nullptr) {
+    CHECK(types[0].as<IncompleteTypeNode>())
+        << "unravel_index: expect input type to be TensorType but get " << types[0];
+    return false;
+  }
+  CHECK(indices->dtype.is_int()) << "indices of unravel_index must be tensor of integer";
+
+  const auto* shape = types[1].as<TensorTypeNode>();
+  if (shape == nullptr) {
+    CHECK(types[1].as<IncompleteTypeNode>())
+        << "unravel_index: expect input type to be TensorType but get " << types[1];
+    return false;
+  }
+  CHECK(indices->dtype.is_int()) << "shape of unravel_index must be tensor of integer";
+
+  Array<IndexExpr> indices_shape;
+  Array<IndexExpr> shape_shape;
+  indices_shape = indices->shape;
+  shape_shape = shape->shape;
+
+  Array<IndexExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (indices_shape.size() != 0) {
+    oshape.push_back(indices_shape[0]);
+  }
+  reporter->Assign(types[2], TensorType(oshape, indices->dtype));
+  return true;
+}
+
+Array<te::Tensor> UnRavelIndexCompute(const Attrs& attrs,
+                                      const Array<te::Tensor>& inputs,
+                                      const Type& out_type) {
+  return Array<te::Tensor>{topi::unravel_index(inputs[0], inputs[1])};
+}
+
+Expr MakeUnRavelIndex(Expr data, Expr shape) {
+  static const Op& op = Op::Get("unravel_index");
+  return CallNode::make(op, {data, shape}, Attrs(), {});
+}
+
+TVM_REGISTER_GLOBAL("relay.op._make.unravel_index").set_body_typed(MakeUnRavelIndex);
+
+RELAY_REGISTER_OP("unravel_index")
+    .describe(
+        R"code(Converts a flat index or array of flat indices into a tuple of coordinate arrays.
+Example::
+    -   unravel_index([22, 41, 37], (7, 6)) = [[3, 6, 6], [4, 5, 1]]
+)code" TVM_ADD_FILELINE)
+    .set_num_inputs(2)
+    .set_support_level(3)
+    .add_type_rel("UnRavelIndexRel", UnRavelIndexRel)
+    .set_attr<FTVMCompute>("FTVMCompute", UnRavelIndexCompute)
+    .set_attr<TOpPattern>("TOpPattern", kInjective);
+
 
 Review comment:
   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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on issue #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
masahi commented on issue #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#issuecomment-602319044
 
 
   Thanks @maheshambule @jwfromm 

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] masahi commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator

Posted by GitBox <gi...@apache.org>.
masahi commented on a change in pull request #5082: [Relay, Topi] [TF, MXNet] Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#discussion_r395434850
 
 

 ##########
 File path: topi/include/topi/transform.h
 ##########
 @@ -232,6 +232,52 @@ inline Tensor reshape(const Tensor& x,
   }
 }
 
+/*!
+ * \brief Converts a flat index or array of flat indices into a tuple of coordinate arrays
+ *
+ * \param x The input tensor having indices.
+ * \param shape The shape tensor
+ * \param name The name of the operation
+ * \param tag The tag to mark the operation
+ *
+ * \return A Tensor of coordinate arrays.
+ */
+
+inline Tensor unravel_index(const Tensor& x, const Tensor& shape, std::string name = "T_unravel",
+                            std::string tag = kInjective) {
+  auto x_shape = x->shape;
+  auto shape_shape = shape->shape;
+
+  Array<PrimExpr> oshape;
+  oshape.push_back(shape_shape[0]);
+  if (x_shape.size() != 0) {
+    oshape.push_back(x_shape[0]);
+  }
+
+  return compute(oshape,
+                 [&](const Array<Var>& indices) {
+                   auto i = indices[0];
+                   std::vector<PrimExpr> indices_divs;
+                   PrimExpr ret = 0;
+                   PrimExpr cur_val = 0;
+                   PrimExpr index_val = 0;
+
+                   if (x_shape.size() != 0) {
+                     index_val = x[indices[1]];
+                   } else {
+                     index_val = x();
+                   }
+                   indices_divs.push_back(index_val);
+                   for (int v = GetConstInt(shape_shape[0]) - 1; v >= 0; --v) {
+                     ret = tvm::if_then_else(i == v, indexmod(indices_divs.back(), shape[v]), ret);
+                     cur_val = indexdiv(indices_divs.back(), shape[v]);
+                     indices_divs.push_back(cur_val);
+                   }
+                   return ret;
+                 },
+                 name, tag);
+}
+
 
 Review comment:
   Rewrite the above with
   
   ```
     auto func = [&](const Array<Var>& indices) {
       auto i = indices[0];
       std::vector<PrimExpr> indices_divs;
       PrimExpr ret = 0;
       PrimExpr cur_val = 0;
       PrimExpr index_val = 0;
   
       if (x_shape.size() != 0) {
         index_val = x[indices[1]];
       } else {
         index_val = x();
       }
       indices_divs.push_back(index_val);
       for (int v = GetConstInt(shape_shape[0]) - 1; v >= 0; --v) {
         ret = tvm::if_then_else(i == v, indexmod(indices_divs.back(), shape[v]), ret);
         cur_val = indexdiv(indices_divs.back(), shape[v]);
         indices_divs.push_back(cur_val);
       }
       return ret;
     };
   
     return compute(oshape, func, name, tag);
   ```
   
   It is generally a good idea to keep indentation level as low as possible.

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


With regards,
Apache Git Services

[GitHub] [incubator-tvm] maheshambule commented on issue #5082: [Relay, Topi] [TF, MXNet]Unravel Index operator

Posted by GitBox <gi...@apache.org>.
maheshambule commented on issue #5082: [Relay, Topi] [TF, MXNet]Unravel Index operator
URL: https://github.com/apache/incubator-tvm/pull/5082#issuecomment-600037169
 
 
   cc: @kevinthesun, @jwfromm, @masahi  Please help in reviewing.

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


With regards,
Apache Git Services