You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mxnet.apache.org by GitBox <gi...@apache.org> on 2020/01/07 06:23:11 UTC

[GitHub] [incubator-mxnet] Tommliu opened a new pull request #17234: Op Quantile/Percentile [Numpy]

Tommliu opened a new pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234
 
 
    ## Description ##
   Numpy operator Quantile and Percentile.   @#16896
    
   ## Checklist ##
   ### Essentials ###
   Please feel free to remove inapplicable items for your PR.
   - [ ] The PR title starts with [MXNET-$JIRA_ID], where $JIRA_ID refers to the relevant [JIRA issue](https://issues.apache.org/jira/projects/MXNET/issues) created (except PRs with tiny changes)
   - [ ] Changes are complete (i.e. I finished coding on this PR)
   - [ ] All changes have test coverage:
   - Unit tests are added for small changes to verify correctness (e.g. adding a new operator)
   - Nightly tests are added for complicated/long-running ones (e.g. changing distributed kvstore)
   - Build tests will be added for build configuration changes (e.g. adding a new build option with NCCL)
   - [ ] Code is well-documented: 
   - For user-facing API changes, API doc string has been updated. 
   - For new C++ functions in header files, their functionalities and arguments are documented. 
   - For new examples, README.md is added to explain the what the example does, the source of the dataset, expected performance on test set and reference to the original paper if applicable
   - Check the API doc at https://mxnet-ci-doc.s3-accelerate.dualstack.amazonaws.com/PR-$PR_ID/$BUILD_ID/index.html
   - [ ] To the best of my knowledge, examples are either not affected by this change, or have been fixed to be compatible with this change
   
   ### Changes ###
   - [ ] Feature1, tests, (and when applicable, API doc)
   - [ ] Feature2, tests, (and when applicable, API doc)
   
   ## Comments ##
   - If this change is a backward incompatible change, why must this change be made.
   - Interesting edge cases to note 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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364656365
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
+
+    float idx = q[q_idx] * (t_shape[NDim-1]-1) / 100.0;
+    int integral_idx = -1;
+    if (interpolation == percentile_enum::kLower) {
+      integral_idx = floor(idx);
+    } else if (interpolation == percentile_enum::kHigher) {
+      integral_idx = ceil(idx);
+    } else if (interpolation == percentile_enum::kMidpoint) {
+      idx = (floor(idx) + ceil(idx)) / 2;
+    } else if (interpolation == percentile_enum::kNearest) {
+      integral_idx = round(idx);
+    }
+
+    if (integral_idx >= 0) {
+      t_coord[NDim-1] = integral_idx;
+      size_t t_idx = ravel(t_coord, t_shape);
+      out[i] = (OType) a_sort[t_idx];
+    } else {
+      int idx_below = floor(idx);
+      int idx_above = idx_below + 1;
+      idx_above = idx_above > t_shape[NDim-1] - 1 ? t_shape[NDim-1] - 1 : idx_above;
+      float weight_above = idx - idx_below;
+      float weight_below = 1 - weight_above;
+      t_coord[NDim-1] = idx_below;
+      size_t t_idx1 = ravel(t_coord, t_shape);
+      size_t t_idx2 = t_idx1 + (idx_above - idx_below);
+      OType x1 = (OType) a_sort[t_idx1] * weight_below;
+      OType x2 = (OType) a_sort[t_idx2] * weight_above;
+      out[i] = x1 + x2;
+    }
+  }
+};
+
+inline bool NumpyPercentileShape(const nnvm::NodeAttrs& attrs,
 
 Review comment:
   please move `Shape` and `Type` functions to `.cc` file.

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364659372
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
 
 Review comment:
   nvm, I mistook r with t

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364660646
 
 

 ##########
 File path: src/operator/tensor/ordering_op-inl.h
 ##########
 @@ -558,6 +558,202 @@ void TopKImpl(const RunContext &ctx,
   }
 }
 
+template<typename xpu, typename DType>
+size_t TopK_Workspace_Cal(const TBlob& src,
+                          const TopKParam& param,
+                          size_t *temp_size_ptr) {
+  using namespace mshadow;
+  using namespace mshadow::expr;
+
+  size_t batch_size = 0;
+  size_t temp_size;
+  index_t element_num = 0;  // number of batches + the size of each batch
+  int axis = 0;
+  bool do_transpose = false;
+  bool is_ascend = false;
+  index_t k = 0;
+  size_t alignment = std::max(sizeof(DType), sizeof(index_t));
+  mxnet::TShape target_shape;
+  ParseTopKParam(src.shape_, param,
+                 &target_shape, &batch_size, &element_num, &axis, &k, &do_transpose, &is_ascend);
+
+  // Temp space needed by the full sorts.
+  temp_size = std::max(
+      mxnet::op::SortByKeyWorkspaceSize<index_t, DType, xpu>(src.Size()),
+      mxnet::op::SortByKeyWorkspaceSize<DType, index_t, xpu>(src.Size()));
+
+  temp_size = std::max(temp_size,
+      mxnet::op::SortByKeyWorkspaceSize<index_t, index_t, xpu>(src.Size()));
+  // Additional temp space for gpu full sorts for batch ids.
+  temp_size += PadBytes(sizeof(index_t) * src.Size(), alignment);
+  // Temp space for cpu sorts.
+  temp_size = std::max(temp_size, sizeof(DType) * src.Size());
+  *temp_size_ptr = temp_size;
+
+  size_t workspace_size = temp_size + PadBytes(sizeof(DType) * src.Size(), alignment)
+                                    + PadBytes(sizeof(index_t) * src.Size(), alignment);
+  if (param.ret_typ == topk_enum::kReturnMask) {
+    workspace_size += PadBytes(sizeof(index_t) * batch_size * k, alignment);
+  }
+  return workspace_size;
+}
+
+template<typename xpu, typename DType, typename IDType>
+void TopK_Workspace_Impl(const RunContext &ctx,
 
 Review comment:
   `TopKImplwithWorkspace`

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364657942
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
+
+    float idx = q[q_idx] * (t_shape[NDim-1]-1) / 100.0;
+    int integral_idx = -1;
+    if (interpolation == percentile_enum::kLower) {
+      integral_idx = floor(idx);
+    } else if (interpolation == percentile_enum::kHigher) {
+      integral_idx = ceil(idx);
+    } else if (interpolation == percentile_enum::kMidpoint) {
+      idx = (floor(idx) + ceil(idx)) / 2;
+    } else if (interpolation == percentile_enum::kNearest) {
+      integral_idx = round(idx);
+    }
+
+    if (integral_idx >= 0) {
+      t_coord[NDim-1] = integral_idx;
+      size_t t_idx = ravel(t_coord, t_shape);
+      out[i] = (OType) a_sort[t_idx];
+    } else {
+      int idx_below = floor(idx);
+      int idx_above = idx_below + 1;
+      idx_above = idx_above > t_shape[NDim-1] - 1 ? t_shape[NDim-1] - 1 : idx_above;
+      float weight_above = idx - idx_below;
+      float weight_below = 1 - weight_above;
+      t_coord[NDim-1] = idx_below;
+      size_t t_idx1 = ravel(t_coord, t_shape);
+      size_t t_idx2 = t_idx1 + (idx_above - idx_below);
+      OType x1 = (OType) a_sort[t_idx1] * weight_below;
 
 Review comment:
   better use something like `static_cast<OType>(...)` instead.

----------------------------------------------------------------
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-mxnet] haojin2 commented on issue #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on issue #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#issuecomment-574893569
 
 
   address the sanity issues: http://jenkins.mxnet-ci.amazon-ml.com/job/mxnet-validation/job/sanity/job/PR-17234/5/display/redirect

----------------------------------------------------------------
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-mxnet] haojin2 merged pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 merged pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234
 
 
   

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364658195
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
+
+    float idx = q[q_idx] * (t_shape[NDim-1]-1) / 100.0;
+    int integral_idx = -1;
+    if (interpolation == percentile_enum::kLower) {
+      integral_idx = floor(idx);
+    } else if (interpolation == percentile_enum::kHigher) {
+      integral_idx = ceil(idx);
+    } else if (interpolation == percentile_enum::kMidpoint) {
+      idx = (floor(idx) + ceil(idx)) / 2;
+    } else if (interpolation == percentile_enum::kNearest) {
+      integral_idx = round(idx);
+    }
+
+    if (integral_idx >= 0) {
+      t_coord[NDim-1] = integral_idx;
+      size_t t_idx = ravel(t_coord, t_shape);
+      out[i] = (OType) a_sort[t_idx];
+    } else {
+      int idx_below = floor(idx);
+      int idx_above = idx_below + 1;
+      idx_above = idx_above > t_shape[NDim-1] - 1 ? t_shape[NDim-1] - 1 : idx_above;
+      float weight_above = idx - idx_below;
+      float weight_below = 1 - weight_above;
+      t_coord[NDim-1] = idx_below;
+      size_t t_idx1 = ravel(t_coord, t_shape);
+      size_t t_idx2 = t_idx1 + (idx_above - idx_below);
+      OType x1 = (OType) a_sort[t_idx1] * weight_below;
 
 Review comment:
   Or at least do `OType(...)`

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364658691
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
 
 Review comment:
   better pre-compute this result outside the kernel, doing it on GPU threads could be too expensive.

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r369860563
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,272 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
+
+    float idx = q[q_idx] * (t_shape[NDim-1]-1) / 100.0;
+    int integral_idx = -1;
+    if (interpolation == percentile_enum::kLower) {
+      integral_idx = floor(idx);
+    } else if (interpolation == percentile_enum::kHigher) {
+      integral_idx = ceil(idx);
+    } else if (interpolation == percentile_enum::kMidpoint) {
+      idx = (floor(idx) + ceil(idx)) / 2;
+    } else if (interpolation == percentile_enum::kNearest) {
+      integral_idx = round(idx);
+    }
+
+    if (integral_idx >= 0) {
+      t_coord[NDim-1] = integral_idx;
+      size_t t_idx = ravel(t_coord, t_shape);
+      out[i] = static_cast<OType> (a_sort[t_idx]);
+    } else {
+      int idx_below = floor(idx);
+      int idx_above = idx_below + 1;
+      idx_above = idx_above > t_shape[NDim-1] - 1 ? t_shape[NDim-1] - 1 : idx_above;
+      float weight_above = idx - idx_below;
+      float weight_below = 1 - weight_above;
+      t_coord[NDim-1] = idx_below;
+      size_t t_idx1 = ravel(t_coord, t_shape);
+      size_t t_idx2 = t_idx1 + (idx_above - idx_below);
+      OType x1 = static_cast<OType>(a_sort[t_idx1] * weight_below);
+      OType x2 = static_cast<OType>(a_sort[t_idx2] * weight_above);
+      out[i] = x1 + x2;
+    }
+  }
+};
+
+template<typename QType, typename xpu>
+bool CheckInvalidInput(mshadow::Stream<xpu> *s,
+                       const QType *data,
+                       const size_t& data_size,
+                       char* is_valid_ptr);
+
+template<typename xpu>
+void NumpyPercentileForward(const nnvm::NodeAttrs& attrs,
+                            const OpContext &ctx,
+                            const std::vector<TBlob> &inputs,
+                            const std::vector<OpReqType> &req,
+                            const std::vector<TBlob> &outputs) {
+  if (req[0] == kNullOp) return;
+  using namespace mxnet;
+  using namespace mxnet_op;
+  CHECK_EQ(inputs.size(), 2U);
+  CHECK_EQ(outputs.size(), 1U);
+
+  Stream<xpu> *s = ctx.get_stream<xpu>();
+  const TBlob &data = inputs[0];
+  const TBlob &percentile = inputs[1];
+  const TBlob &out = outputs[0];
+  const NumpyPercentileParam& param = nnvm::get<NumpyPercentileParam>(attrs.parsed);
+  const int interpolation = param.interpolation;
+  dmlc::optional<mxnet::Tuple<int>> axis = param.axis;
+
+  auto small = NumpyReduceAxesShapeImpl(data.shape_, axis, false);
+
+  TShape r_shape;
+  r_shape = TShape(small.ndim()+1, 1);
+  r_shape[0] = percentile.Size();
+  for (int i = 1; i < r_shape.ndim(); ++i) {
+    r_shape[i] = small[i-1];
+  }
+  // Origin axes
+  TShape axes;
+  if (!axis.has_value()) {
+    axes = TShape(data.shape_.ndim(), 1);
+    for (int i = 0; i < data.shape_.ndim(); ++i) {
+      axes[i] = i;
+    }
+  } else {
+    auto axis_tuple = axis.value();
+    axes = TShape(axis_tuple.ndim(), 1);
+    for (int i = 0; i < axis_tuple.ndim(); ++i) {
+      if (axis_tuple[i] < 0) {
+        axes[i] = axis_tuple[i] + data.shape_.ndim();
+      } else {
+        axes[i] = axis_tuple[i];
+      }
+    }
+  }
+  // Transpose the axes
+  TShape t_axes(data.shape_.ndim(), 1);
+  int j = 0;
+  for (int i = 0; i < t_axes.ndim(); ++i) {
+    bool red = false;
+    for (int k = 0; k < axes.ndim(); ++k) {
+      if (axes[k] == i) {
+        red = true;
+      }
+    }
+    if (!red) {
+      t_axes[j] = i;
+      j++;
+    }
+  }
+  for (int jj = j; jj < t_axes.ndim(); ++jj) {
+    t_axes[jj] = axes[jj-j];
+  }
+  // Transpose Shape with reduced dims at dim [-1]
+  TShape t_shape(small.ndim()+1, 1);
+  for (int i = 0; i < small.ndim(); ++i) {
+    t_shape[i] = small[i];
+  }
+  size_t red_size = 1;
+  for (int i = 0; i < axes.ndim(); ++i) {
+    red_size *= data.shape_[axes[i]];
+  }
+  t_shape[t_shape.ndim()-1] = red_size;
+  // Transpose Shape extension
+  TShape t_shape_ex(data.shape_.ndim(), 1);
+  for (int i = 0; i < data.shape_.ndim(); ++i) {
+    t_shape_ex[i] = data.shape_[t_axes[i]];
+  }
+  TopKParam topk_param = TopKParam();
+  topk_param.axis = dmlc::optional<int>(-1);
+  topk_param.is_ascend = true;
+  topk_param.k = 0;
+  topk_param.ret_typ = topk_enum::kReturnValue;
+
+  MSHADOW_TYPE_SWITCH(data.type_flag_, DType, {
+    size_t temp_size;  // Used by Sort
+    size_t topk_workspace_size = TopKWorkspaceSize<xpu, DType>(data, topk_param, &temp_size);
+
+    size_t temp_data_size = data.Size() * sizeof(DType);
+    size_t idx_size = data.Size() * sizeof(index_t);
+    size_t temp_mem_size = 2 * temp_data_size + idx_size;
+    size_t workspace_size = topk_workspace_size * 2 + temp_mem_size + 8;
+
+    Tensor<xpu, 1, char> temp_mem =
+      ctx.requested[0].get_space_typed<xpu, 1, char>(Shape1(workspace_size), s);
+
+    char* workspace_curr_ptr = temp_mem.dptr_;
+    DType* trans_ptr, *sort_ptr;
+    index_t* idx_ptr;
+
+    char* is_valid_ptr = reinterpret_cast<char*>(workspace_curr_ptr);
+    MSHADOW_TYPE_SWITCH(percentile.type_flag_, QType, {
+      bool is_valid = CheckInvalidInput<QType, xpu>(s, percentile.dptr<QType>(),
+                                                    percentile.Size(), is_valid_ptr);
+      CHECK(is_valid)<< "ValueError: percentile exceeds the valid range";
+    })   // check the invalid percentile
+    workspace_curr_ptr += 8;
+
+    if (sizeof(DType) >= sizeof(index_t)) {
+      trans_ptr = reinterpret_cast<DType*>(workspace_curr_ptr);
+      sort_ptr = reinterpret_cast<DType*>(workspace_curr_ptr + temp_data_size);
+      idx_ptr = reinterpret_cast<index_t*>(workspace_curr_ptr + 2 * temp_data_size);
+    } else {
+      idx_ptr = reinterpret_cast<index_t*>(workspace_curr_ptr);
+      trans_ptr = reinterpret_cast<DType*>(workspace_curr_ptr + idx_size);
+      sort_ptr = reinterpret_cast<DType*>(workspace_curr_ptr + temp_data_size + idx_size);
+    }
+    workspace_curr_ptr += 2 * temp_data_size + idx_size;
+
+    TBlob a_trans = TBlob(trans_ptr, t_shape_ex, xpu::kDevMask);
+    TransposeImpl<xpu>(ctx.run_ctx, data, a_trans, t_axes);
+    TBlob a_sort = TBlob(sort_ptr, t_shape, xpu::kDevMask);
+    TBlob a_idx = TBlob(idx_ptr, t_shape, xpu::kDevMask);
+    std::vector<OpReqType> req_TopK = {kWriteTo, kNullOp};
+    TBlob src = a_trans.reshape(t_shape);
+    std::vector<TBlob> ret = {a_sort, a_idx};
+
+    TopKImplwithWorkspace<xpu, DType, index_t>(ctx.run_ctx, req_TopK, src, ret, topk_param,
+                                               workspace_curr_ptr, temp_size, s);
+    MSHADOW_TYPE_SWITCH(percentile.type_flag_, QType, {
+      MSHADOW_TYPE_SWITCH(out.type_flag_, OType, {
 
 Review comment:
   change this to `MSHADOW_SGL_DBL_TYPE_SWITCH`

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r363908323
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op.cu
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op.cu
+ * \brief GPU Implementation of Numpy-compatible percentile
+*/
+
+#include "np_percentile_op-inl.h"
+
+namespace mxnet {
+namespace op {
+
+struct is_valid_check {
+    template<typename QType>
+    MSHADOW_XINLINE static void Map(int i, char* invalid_ptr, const QType* data) {
+        if (data[i] < 0.0 || data[i] > 100) *invalid_ptr = 1;
+    }
 
 Review comment:
   2-space indentations, same for the function below.

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364658344
 
 

 ##########
 File path: src/operator/numpy/np_percentile_op-inl.h
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.
+ */
+/*!
+ * Copyright (c) 2019 by Contributors
+ * \file np_percentile_op-inl.h
+*/
+
+#ifndef MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+#define MXNET_OPERATOR_NUMPY_NP_PERCENTILE_OP_INL_H_
+
+#include <vector>
+#include "../tensor/ordering_op-inl.h"
+#include "../tensor/matrix_op-inl.h"
+#include "../../common/utils.h"
+#include "../mshadow_op.h"
+#include "../operator_common.h"
+#include "../elemwise_op_common.h"
+#include "np_broadcast_reduce_op.h"
+
+namespace mxnet {
+namespace op {
+
+namespace percentile_enum {
+enum PercentileType {kLinear, kLower, kHigher, kMidpoint, kNearest};
+}  // percentile_enum
+
+struct NumpyPercentileParam : public dmlc::Parameter<NumpyPercentileParam> {
+  dmlc::optional<mxnet::Tuple<int>> axis;
+  int interpolation;
+  bool keepdims;
+  DMLC_DECLARE_PARAMETER(NumpyPercentileParam) {
+    DMLC_DECLARE_FIELD(axis)
+      .set_default(dmlc::optional<mxnet::Tuple<int>>())
+      .describe("Axis or axes along which a sum is performed. The default, axis=None, will sum "
+                "all of the elements of the input array. If axis is negative it counts from the "
+                "last to the first axis.");
+    DMLC_DECLARE_FIELD(interpolation).set_default(percentile_enum::kLinear)
+      .add_enum("linear", percentile_enum::kLinear)
+      .add_enum("lower", percentile_enum::kLower)
+      .add_enum("higher", percentile_enum::kHigher)
+      .add_enum("midpoint", percentile_enum::kMidpoint)
+      .add_enum("nearest", percentile_enum::kNearest)
+      .describe("his optional parameter specifies the interpolation method to use when the"
+                "desired percentile lies between two data points i < j");
+    DMLC_DECLARE_FIELD(keepdims).set_default(false)
+      .describe("If this is set to `True`, the reduced axes are left "
+                "in the result as dimension with size one.");
+  }
+};
+
+template<int NDim>
+struct percentile_take {
+  template<typename DType, typename QType, typename OType>
+  MSHADOW_XINLINE static void Map(int i,
+                                  OType* out,
+                                  const QType* q,
+                                  const DType* a_sort,
+                                  const int interpolation,
+                                  mshadow::Shape<NDim> t_shape,
+                                  mshadow::Shape<NDim> r_shape) {
+    using namespace mshadow;
+    using namespace mxnet_op;
+
+    auto r_coord = unravel(i, r_shape);
+    size_t q_idx = r_coord[0];
+
+    Shape<NDim> t_coord(t_shape);
+
+    for (int j = 0; j < NDim-1; ++j) {
+      t_coord[j] = r_coord[j+1];
+    }
+
+    float idx = q[q_idx] * (t_shape[NDim-1]-1) / 100.0;
+    int integral_idx = -1;
+    if (interpolation == percentile_enum::kLower) {
+      integral_idx = floor(idx);
+    } else if (interpolation == percentile_enum::kHigher) {
+      integral_idx = ceil(idx);
+    } else if (interpolation == percentile_enum::kMidpoint) {
+      idx = (floor(idx) + ceil(idx)) / 2;
+    } else if (interpolation == percentile_enum::kNearest) {
+      integral_idx = round(idx);
+    }
+
+    if (integral_idx >= 0) {
+      t_coord[NDim-1] = integral_idx;
+      size_t t_idx = ravel(t_coord, t_shape);
+      out[i] = (OType) a_sort[t_idx];
+    } else {
+      int idx_below = floor(idx);
+      int idx_above = idx_below + 1;
+      idx_above = idx_above > t_shape[NDim-1] - 1 ? t_shape[NDim-1] - 1 : idx_above;
+      float weight_above = idx - idx_below;
+      float weight_below = 1 - weight_above;
+      t_coord[NDim-1] = idx_below;
+      size_t t_idx1 = ravel(t_coord, t_shape);
+      size_t t_idx2 = t_idx1 + (idx_above - idx_below);
+      OType x1 = (OType) a_sort[t_idx1] * weight_below;
 
 Review comment:
   same for above and below.

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364660488
 
 

 ##########
 File path: src/operator/tensor/ordering_op-inl.h
 ##########
 @@ -558,6 +558,202 @@ void TopKImpl(const RunContext &ctx,
   }
 }
 
+template<typename xpu, typename DType>
+size_t TopK_Workspace_Cal(const TBlob& src,
 
 Review comment:
   Better get rid of the underscores in your function names, follow the convention of all other funcs in the file, you could probably rename to `TopKWorkspaceSize`

----------------------------------------------------------------
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-mxnet] haojin2 commented on issue #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on issue #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#issuecomment-574894032
 
 
   Also, a bad non-ASCII character: 
   ```
   Failure: SyntaxError (Non-ASCII character '\xe2' in file /work/mxnet/python/mxnet/ndarray/numpy/_op.py on line 6050, but no encoding declared; see http://python.org/dev/peps/pep-0263/ for details (_op.py, line 6049))
   ```

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364660946
 
 

 ##########
 File path: tests/python/unittest/test_numpy_interoperability.py
 ##########
 @@ -156,6 +156,20 @@ def _add_workload_diagonal():
     OpArgMngr.add_workload('diagonal', B, offset=1, axis1=0, axis2=2)
     OpArgMngr.add_workload('diagonal', B, 0, 2, 1)
 
+def _add_workload_quantile():
+    x1 = np.arange(8) * 0.5
+    x2 = np.arange(100.)
+    q1 = np.array(0)
+    q2 = np.array(1)
+    q3 = np.array(0.5)
+    q4 = np.array([0, 0.75, 0.25, 0.5, 1.0])
+    
+    OpArgMngr.add_workload('quantile', x1, q1)
+    OpArgMngr.add_workload('quantile', x1, q2)
+    OpArgMngr.add_workload('quantile', x1, q3)
+    OpArgMngr.add_workload('quantile', x2, q4, interpolation="midpoint")
+    OpArgMngr.add_workload('quantile', x2, q4, interpolation="nearest")
+    OpArgMngr.add_workload('quantile', x2, q4, interpolation="lower")
 
 Review comment:
   also add for percentile

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r367163259
 
 

 ##########
 File path: python/mxnet/ndarray/numpy/_op.py
 ##########
 @@ -6023,6 +6023,159 @@ def nonzero(a):
     return tuple([out[i] for i in range(len(out))])
 
 
+@set_module('mxnet.ndarray.numpy')
+def percentile(a, q, axis=None, out=None, overwrite_input=None, interpolation='linear', keepdims=False): # pylint: disable=too-many-arguments
+    """
+    Compute the q-th percentile of the data along the specified axis.
+    Returns the q-th percentile(s) of the array elements.
+
+    Parameters
+    ----------
+    a : array_like
 
 Review comment:
   better use `ndarray`, since `array_like` could also include list, tuple etc.

----------------------------------------------------------------
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-mxnet] haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]

Posted by GitBox <gi...@apache.org>.
haojin2 commented on a change in pull request #17234: Op Quantile/Percentile [Numpy]
URL: https://github.com/apache/incubator-mxnet/pull/17234#discussion_r364661074
 
 

 ##########
 File path: tests/python/unittest/test_numpy_op.py
 ##########
 @@ -5680,6 +5680,49 @@ def test_np_share_memory():
                 assert not op(np.ones((5, 0), dtype=dt), np.ones((0, 3, 0), dtype=adt))
 
 
+@with_seed()
+@use_np
+def test_np_quantile():
+    class TestQuantile(HybridBlock):
+        def __init__(self, axis=None, interpolation='linear', keepdims=False):
+            super(TestQuantile, self).__init__()
+            self._axis = axis
+            self._interpolation = interpolation
+            self._keepdims = keepdims
+
+        def hybrid_forward(self, F, a, q):
+            return F.np.quantile(a, q, axis=self._axis, interpolation=self._interpolation, keepdims=self._keepdims)
+
+    flags = [True, False]
+    interpolation_options = ['linear', 'lower', 'higher', 'nearest', 'midpoint']
+    dtypes = [np.int32, np.int64, np.float16, np.float32, np.float64]
+    qtypes = [np.float32, np.float64]
+    tensor_shapes = [
+        ((2, 3), (), None),
+        ((2, 3, 4, 5), (), 3),
+        ((2, 3, 4), (3,), (0, 2)),
+        ((2, 3, 4), (3,), 1)
+    ]
+    for hybridize, keepdims, (a_shape, q_shape, axis), interpolation, dtype in \
+        itertools.product(flags, flags, tensor_shapes, interpolation_options, dtypes):
+        atol = 3e-4 if dtype == np.float16 else 1e-4
+        rtol = 3e-2 if dtype == np.float16 else 1e-2
+        test_quantile = TestQuantile(axis=axis, interpolation=interpolation, keepdims=keepdims)
+        if hybridize:
+            test_quantile.hybridize()
+        a = np.random.uniform(-10.0, 10.0, size=a_shape).astype(dtype)
+        qtype = random.choice(qtypes)
+        q = np.random.uniform(0, 1.0, size=q_shape).astype(qtype)
+        np_out = _np.quantile(a.asnumpy(), q.asnumpy(), axis=axis, interpolation=interpolation, keepdims=keepdims)
+        mx_out = test_quantile(a, q)
+        assert mx_out.shape == np_out.shape
+        assert_almost_equal(mx_out.asnumpy(), np_out, atol=atol, rtol=rtol)
+
+        mx_out = np.quantile(a, q, axis=axis, interpolation=interpolation, keepdims=keepdims)
+        np_out = _np.quantile(a.asnumpy(), q.asnumpy(), axis=axis, interpolation=interpolation, keepdims=keepdims)
+        assert_almost_equal(mx_out.asnumpy(), np_out, atol=atol, rtol=rtol)
 
 Review comment:
   also add test for percentile, or make this one a compound unit test

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