You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/12/13 13:29:35 UTC

[GitHub] [arrow] mbrobbel opened a new pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

mbrobbel opened a new pull request #11937:
URL: https://github.com/apache/arrow/pull/11937


   Initially I tried to implement this as a `ScalarAggregateFunction` (as suggested in the issue), however given that there is no way to express sensitivity to order, it's currently not possible to correctly implement the `ScalarAggregator::MergeFrom` function. This is now implemented as a `VectorFunction`.
   
   I'm still working on supporting more types:
   - [x] Booleans
   - [x] Signed and unsigned integer types
   - [ ] Floating point types (I'm trying to figure out what I can re-use)
   - [ ] Temporal types
   - [ ] Decimal types
   
   I'll create two follow-up JIRA issues to:
   
   - Add support for UTF8 arrays (via lexicographical order):
   ```
   Input: utf8
   [
     "a",
     "b",
     "c"
   ]
   
   Output: struct<increasing:boolean, strictly_decreasing:boolean, decreasing:boolean, strictly_decreasing:boolean>
   {increasing: true, strictly_increasing: true, decreasing: false, strictly_decreasing: false}
   ```
   
   - Implement this function for list arrays with well-ordered element types:
   ```
   Input: list<uint8>
   [
     [1, 2, 3],
     [3, 3, 2],
     [1, 2, 1, 2]
   ]
   
   Output list<struct<increasing:boolean, strictly_decreasing:boolean, decreasing:boolean, strictly_decreasing:boolean>>
   [
     {increasing: true, strictly_increasing: true, decreasing: false, strictly_decreasing: false},
     {increasing: false, strictly_increasing: false, decreasing: true, strictly_decreasing: false},
     {increasing: false, stirctly_increasing: false, decreasing: false, strictly_decreasing: false}
   ]
   ```
   
   cc @bkietz


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r772452442



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,317 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,

Review comment:
       After reviewing this PR, I think implementations of `IsMonotonicCheck` can be categorized as follows based on `DataType`:
   * have `c_type` (e.g., primitive numeric, datetime, timestamp)
      * `c_type` is floating-point
      * `c_type` is not floating-point
   * do not have `c_type` (binary, string, intervals)
      * custom implementation for `binary` and `string`
      * custom implementation for intervals
   
   There would be at least 4 type-specific implementations of `IsMonotonicCheck`, where the
   the `enable_if` for this case would be of the form (pseudocode) [`enable_if_has_c_type and enable_if_not_floating_point`](https://github.com/apache/arrow/blob/master/cpp/src/arrow/type_traits.h#L748) 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r772452442



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,317 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,

Review comment:
       After reviewing this PR, I think implementations of `IsMonotonicCheck` can be categorized as follows based on `DataType`:
   * have `c_type` (e.g., primitive numeric, datetime, timestamp)
      * `c_type` is floating-point
      * `c_type` is not floating-point
   * do not have `c_type` (binary, string, intervals)
      * custom implementation for `binary` and `string`
      * custom implementation for intervals
   
   There would be at least 4 type-specific implementations of `IsMonotonicCheck`, where the
   the `enable_if` for this case would be of the form (pseudocode) [`enable_if_has_c_type and !floating_point`](https://github.com/apache/arrow/blob/master/cpp/src/arrow/type_traits.h#L748) 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r770988896



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||

Review comment:
       
   @pitrou Should we consider signed zero/Inf here? Not sure if sorting function does it. In any case, consistency is desired and can be resolved in a follow-up JIRA.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce edited a comment on pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#issuecomment-996249724


   Some general comments:
   * `IsMonotonic` needs to be consistent with corresponding [sorting functions](https://arrow.apache.org/docs/cpp/compute.html#sorts-and-partitions), such that `IsMonotonic(input) == (Sort(input) == input)`.
   * Currently `IsMonotonic` outputs a struct describing the monotonic properties of the data. What your thoughts on having a convenience wrapper function that receives `FunctionOptions` with a single requested monotonic behavior? For example, `IsMonotonic(input, MonotonicOptions.StrictlyIncreasing)`. This would provide a more readable API for client code and bypass the need to unpack the `StructScalar` output to check the corresponding monotonic behavior of interest.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#issuecomment-996249724


   Some general comments:
   * `IsMonotonic` needs to be consistent with corresponding [sorting functions](https://arrow.apache.org/docs/cpp/compute.html#sorts-and-partitions), such that `IsMonotonic(input) == (Sort(input) == input)`.
   * Currently `IsMonotonic` outputs a struct describing the monotonic properties of the data. Would it make sense to provide a wrapper function that receives `FunctionOptions` with a single requested monotonic behavior? For example, `IsMonotonic(input, MonotonicOptions.StrictlyIncreasing)`


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] mbrobbel commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
mbrobbel commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771651816



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,35 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    /// Ignore nulls.
+    IGNORE_NULLS,

Review comment:
       I had that initially, but IGNORE caused compilation issues on Windows.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771658198



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,35 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    /// Ignore nulls.
+    IGNORE_NULLS,

Review comment:
       Well, nevermind.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] mbrobbel commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
mbrobbel commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771266725



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {
+    return Status::Invalid("IsMonotonic expects a single datum (array) as input");
+  }
+
+  // Safety:
+  // - Made sure there is at least one input datum.
+  Datum input = batch[0];
+
+  // Validate input datum type (useful for direct invocation only).
+  if (!input.is_array()) {
+    return Status::Invalid("IsMonotonic expects array datum as input");
+  }
+
+  // Safety:
+  // - Made sure that the input datum is an array.
+  const std::shared_ptr<ArrayData>& array_data = input.array();
+  ArrayType array(array_data);
+
+  // Return early if there are NaNs, zero elements or one element in the array.
+  // And return early if there are only nulls.
+  if (array.length() <= 1 || array.null_count() == array.length()) {
+    if (std::any_of(array.begin(), array.end(), isnan<DataType>)) {

Review comment:
       What would that look like?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771275206



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||

Review comment:
       Sorting doesn't, AFAIR. Signed zeros are considered equal, I'm not sure there's any particular reason to deviate from that (what are the use cases for this kernel?).




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r770940279



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;

Review comment:
       Ideally, `isnan()` should only be used in floating-point-enabled functions, so maybe you can function overload (using enable-if magic) the code blocks that use `isnan()` in a more generic manner.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce edited a comment on pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#issuecomment-996249724


   Some general comments:
   * `IsMonotonic` needs to be consistent with corresponding [sorting functions](https://arrow.apache.org/docs/cpp/compute.html#sorts-and-partitions), such that `IsMonotonic(input) == (Sort(input) == input)`.
   * Currently `IsMonotonic` outputs a struct describing the monotonic properties of the data. What are your thoughts on having a convenience wrapper function that receives `FunctionOptions` with a single requested monotonic behavior? For example, `IsMonotonic(input, MonotonicOptions.StrictlyIncreasing)`. This would provide a more readable API for client code and bypass the need to unpack the `StructScalar` output to check the corresponding monotonic behavior of interest.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r770964847



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {
+    return Status::Invalid("IsMonotonic expects a single datum (array) as input");
+  }
+
+  // Safety:
+  // - Made sure there is at least one input datum.
+  Datum input = batch[0];
+
+  // Validate input datum type (useful for direct invocation only).
+  if (!input.is_array()) {
+    return Status::Invalid("IsMonotonic expects array datum as input");
+  }
+
+  // Safety:
+  // - Made sure that the input datum is an array.
+  const std::shared_ptr<ArrayData>& array_data = input.array();
+  ArrayType array(array_data);
+
+  // Return early if there are NaNs, zero elements or one element in the array.
+  // And return early if there are only nulls.
+  if (array.length() <= 1 || array.null_count() == array.length()) {
+    if (std::any_of(array.begin(), array.end(), isnan<DataType>)) {
+      return IsMonotonicOutput(false, false, false, false, out);
+    } else {
+      // It is strictly increasing if there are zero or one elements or when nulls are
+      // ignored.
+      bool strictly =
+          array.length() <= 1 ||
+          options.null_handling == IsMonotonicOptions::NullHandling::IGNORE_NULLS;
+      return IsMonotonicOutput(true, strictly, true, strictly, out);
+    }
+  }
+
+  // Set null value based on option.
+  const CType null_value =
+      options.null_handling == IsMonotonicOptions::NullHandling::USE_MIN_VALUE
+          ? min<DataType>()
+          : max<DataType>();
+
+  bool increasing = true, strictly_increasing = true, decreasing = true,
+       strictly_decreasing = true;
+
+  // Safety:
+  // - Made sure that the length is at least 2 above.
+  for (auto a = array.begin(), b = ++array.begin(); b != array.end();) {
+    auto current = *a;
+    auto next = *b;
+
+    // Handle nulls.
+    if (options.null_handling == IsMonotonicOptions::NullHandling::IGNORE_NULLS) {
+      // Forward both iterators to search for a non-null value. The loop exit
+      // condition prevents reading past the end.
+      if (!current.has_value()) {
+        ++a;
+        ++b;
+        continue;
+      }
+      // Once we have a value for current we should also make sure that next has a
+      // value. The loop exit condition prevents reading past the end.
+      if (!next.has_value()) {
+        ++b;
+        continue;
+      }
+    } else {
+      // Based on the function options set null values to min/max.
+      current = current.value_or(null_value);
+      next = next.value_or(null_value);

Review comment:
       Are these local copies being modified, correct? An Boolean function should not modify its inputs.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] bkietz commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r768106150



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,29 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    // Ignore nulls.

Review comment:
       Please use /// to allow doxygen to detect docstrings
   ```suggestion
       /// Ignore nulls.
   ```

##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,29 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    // Ignore nulls.
+    IGNORE,
+    // Use min value of element type as the value of nulls.
+    MIN,
+    // Use max value of element type as the value of nulls.
+    MAX,
+  };
+
+  explicit IsMonotonicOptions(NullHandling null_handling = IGNORE,
+                              EqualOptions equal_options = EqualOptions::Defaults());
+  constexpr static char const kTypeName[] = "IsMonotonicOptions";
+  static IsMonotonicOptions Defaults() { return IsMonotonicOptions(); }
+
+  // Define how nulls are handled.
+  NullHandling null_handling;
+  // Options for equality comparisons. Used for floats.
+  EqualOptions equal_options;

Review comment:
       Instead of reusing this here, please add a separate enumeration for nan handling (`nans_equal` seems to be the only field you're using here). `is_monotonic_to_within_tolerance` sounds like a separate (floating-point-only) function.
   
   (Adding to the confusion here is: based on naming, `EqualOptions` looks like a subclass of `FunctionOptions` but isn'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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] mbrobbel commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
mbrobbel commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771264722



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {

Review comment:
       Directly invocation of this kernel is not possible through the public API, however internally this function could be invoked directly and skip those checks. @bkietz what do you suggest?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771604225



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,35 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    /// Ignore nulls.
+    IGNORE_NULLS,

Review comment:
       Based on the other enum names, use only `IGNORE`, since `enum NullHandling` already specifies this is for nulls.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] github-actions[bot] commented on pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#issuecomment-992486275


   https://issues.apache.org/jira/browse/ARROW-1569


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771598458



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {

Review comment:
       IMHO, this check is trivial, it doesn't hurt having 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] mbrobbel commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
mbrobbel commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r771354239



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,35 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    /// Ignore nulls.
+    IGNORE_NULLS,
+    /// Use min value of element type as the value of nulls.
+    /// -Inf for floating point numbers.
+    USE_MIN_VALUE,
+    /// Use max value of element type as the value of nulls.
+    /// Inf for floating point numbers.
+    USE_MAX_VALUE
+  };

Review comment:
       Yes, I agree that a sort before invoking this kernel should result in true for the corresponding check. However I feel the null handling variants are a bit confusing: `AtStart` defines `NaN > null` and `AtEnd` defines `NaN < null`. Also, the sorting kernel can ignore equality, but this kernels considers it to check if values are unique (strictly increasing/decreasing).
   
   I think if we want to allow users to define order of unordered values (both for sorting and this kernel) we need something like this:
   ```c++
   bool compare_nulls = false; // default: any null results in false outputs (or error in case of sort)
   bool compare_nans = false; // default: any nan results in false outputs (or error in case of sort)
   
   // these are not needed when sorting
   bool nulls_equal = false; // when nulls are compared, are they considered equal?
   bool nans_equal = false; // when nans are compared, are they considered equal?
   
   // when both nulls and nans are compared
   enum Ordering { Less, Equal, Greater }
   Ordering nan_compared_with_null; // when comparing nulls and nans, what ordering should be used?
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] mbrobbel commented on pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
mbrobbel commented on pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#issuecomment-996683557


   >     * `IsMonotonic` needs to be consistent with corresponding [sorting functions](https://arrow.apache.org/docs/cpp/compute.html#sorts-and-partitions), such that `IsMonotonic(input) == (Sort(input) == input)`.
   
   I agree.
   
   >     * Currently `IsMonotonic` outputs a struct describing the monotonic properties of the data. What your thoughts on having a convenience wrapper function that receives `FunctionOptions` with a single requested monotonic behavior? For example, `IsMonotonic(input, MonotonicOptions.StrictlyIncreasing)`. This would provide a more readable API for client code and bypass the need to unpack the `StructScalar` output to check the corresponding monotonic behavior of interest.
   
   I initially set it up like that but @bkietz suggested to output a struct scalar 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r770940279



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;

Review comment:
       Ideally, `isnan()` should only be used in floating-point-enabled functions, so maybe you can function overload (using enable-if) the code blocks that use `isnan()` too generically.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {

Review comment:
       AFAIK, the number of arguments to a function are validated in the compute layer mechanism. When the [`IsMonotonic` function is registered, it specifies a single input argument](https://github.com/apache/arrow/pull/11937/files#diff-011285d7622074d16c9075007d23066a4fbb31fd9bb95c596dab315ed0ca038aR273), so this should be already guaranteed.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||

Review comment:
       There exists support for [floating-point comparisons](https://github.com/apache/arrow/blob/master/cpp/src/arrow/compare.cc#L74-L89), maybe you can reuse this here.
   Also, this check is not considering sign bit for special cases such as: signed zeros and signed Inf.
   Ex. `{ -0.0, 0.0 } != { 0.0, -0.0 }`.

##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -188,6 +188,35 @@ class ARROW_EXPORT PartitionNthOptions : public FunctionOptions {
   NullPlacement null_placement;
 };
 
+/// \brief Options for IsMonotonic
+class ARROW_EXPORT IsMonotonicOptions : public FunctionOptions {
+ public:
+  enum NullHandling {
+    /// Ignore nulls.
+    IGNORE_NULLS,
+    /// Use min value of element type as the value of nulls.
+    /// -Inf for floating point numbers.
+    USE_MIN_VALUE,
+    /// Use max value of element type as the value of nulls.
+    /// Inf for floating point numbers.
+    USE_MAX_VALUE
+  };

Review comment:
       [Ordering of nulls and NaNs were also discussed in the sorting function](https://github.com/apache/arrow/pull/11212/files#diff-d8e3089c085273e425a5106e77566f2e1348c483a79eeb544a4c9ee0e8327f34R89-R96). I would expect that `IsMonotonic` and sorting function are consistent. That is, if a sort operation is performed first, then the corresponding `IsMonotonic` should result in true.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {
+    return Status::Invalid("IsMonotonic expects a single datum (array) as input");
+  }
+
+  // Safety:
+  // - Made sure there is at least one input datum.
+  Datum input = batch[0];
+
+  // Validate input datum type (useful for direct invocation only).
+  if (!input.is_array()) {
+    return Status::Invalid("IsMonotonic expects array datum as input");
+  }
+
+  // Safety:
+  // - Made sure that the input datum is an array.
+  const std::shared_ptr<ArrayData>& array_data = input.array();
+  ArrayType array(array_data);
+
+  // Return early if there are NaNs, zero elements or one element in the array.

Review comment:
       Not sure the NaN comment applies here.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {
+    return Status::Invalid("IsMonotonic expects a single datum (array) as input");
+  }
+
+  // Safety:
+  // - Made sure there is at least one input datum.
+  Datum input = batch[0];
+
+  // Validate input datum type (useful for direct invocation only).
+  if (!input.is_array()) {
+    return Status::Invalid("IsMonotonic expects array datum as input");
+  }
+
+  // Safety:
+  // - Made sure that the input datum is an array.
+  const std::shared_ptr<ArrayData>& array_data = input.array();
+  ArrayType array(array_data);
+
+  // Return early if there are NaNs, zero elements or one element in the array.
+  // And return early if there are only nulls.
+  if (array.length() <= 1 || array.null_count() == array.length()) {
+    if (std::any_of(array.begin(), array.end(), isnan<DataType>)) {

Review comment:
       Everything in this functions seems general enough to handle most data types, except for `isnan`. Maybe specialize this code block.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r770941630



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {

Review comment:
       AFAIK, the number of arguments to a function are validated in the compute layer mechanism. When the [`IsMonotonic` function is registered, it specifies a single input argument](https://github.com/apache/arrow/pull/11937/files#diff-011285d7622074d16c9075007d23066a4fbb31fd9bb95c596dab315ed0ca038aR273), so this should already be guaranteed.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r772434811



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,317 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {
+    return Status::Invalid("IsMonotonic expects a single datum (array) as input");
+  }
+
+  // Safety:
+  // - Made sure there is at least one input datum.
+  Datum input = batch[0];
+
+  // Validate input datum type (useful for direct invocation only).
+  if (!input.is_array()) {
+    return Status::Invalid("IsMonotonic expects array datum as input");
+  }
+
+  // Safety:
+  // - Made sure that the input datum is an array.
+  const std::shared_ptr<ArrayData>& array_data = input.array();

Review comment:
       You can use `auto array_data = input.array()`.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,317 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,

Review comment:
       After reviewing this PR, I think implementations of `IsMonotonicCheck` can be categorized as follows based on `DataType`:
   * have `c_type` (e.g., primitive numeric, datetime, timestamp)
      * `c_type` is floating-point
      * `c_type` is not floating-point
   * do not have `c_type` (binary, string, intervals)
      * custom implementation for `binary` and `string`
      * custom implementation for intervals
   
   There would be at least 4 type-specific implementations of `IsMonotonicCheck`, where the
   the `enable_if` for this case would be of the form (pseudocode) [`enable_if_has_c_type and !not_floating_point`](https://github.com/apache/arrow/blob/master/cpp/src/arrow/type_traits.h#L748) 

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,317 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>

Review comment:
       Nit: `ArrowType` would be more appropriate than `DataType`.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,317 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {

Review comment:
       Since this version of `IsMonotonic` requires `DataType` having a `c_type`, this needs to be guarded with [`enable_if_has_c_type`](https://github.com/apache/arrow/blob/master/cpp/src/arrow/type_traits.h#L748). You would need to make other versions for `binary/string` and interval types.

##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> min() {
+  return -std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::infinity();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> min() {
+  return std::numeric_limits<typename DataType::c_type>::min();
+}
+
+template <typename DataType>
+constexpr enable_if_not_floating_point<DataType, typename DataType::c_type> max() {
+  return std::numeric_limits<typename DataType::c_type>::max();
+}
+
+template <typename DataType>
+Status IsMonotonic(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  using ArrayType = typename TypeTraits<DataType>::ArrayType;
+  using CType = typename TypeTraits<DataType>::CType;
+
+  auto options = IsMonotonicState::Get(ctx);
+
+  // Check batch size
+  if (batch.values.size() != 1) {
+    return Status::Invalid("IsMonotonic expects a single datum (array) as input");
+  }
+
+  // Safety:
+  // - Made sure there is at least one input datum.
+  Datum input = batch[0];
+
+  // Validate input datum type (useful for direct invocation only).
+  if (!input.is_array()) {
+    return Status::Invalid("IsMonotonic expects array datum as input");
+  }
+
+  // Safety:
+  // - Made sure that the input datum is an array.
+  const std::shared_ptr<ArrayData>& array_data = input.array();
+  ArrayType array(array_data);
+
+  // Return early if there are NaNs, zero elements or one element in the array.
+  // And return early if there are only nulls.
+  if (array.length() <= 1 || array.null_count() == array.length()) {
+    if (std::any_of(array.begin(), array.end(), isnan<DataType>)) {

Review comment:
       After more careful thought, what we want is for only floating-point types to do the `std::any()` check. You can use [`TypeTraits` for those that have a `c_type` defined](https://github.com/apache/arrow/blob/master/cpp/src/arrow/type_traits.h#L88-L300) to get a `type_id` variable which can be checked during runtime.
   ```c++
   if (array.length() <= 1 || array.null_count() == array.length()) {
     auto type_id = TypeTraits<DataType>::type_singleton();
     if (!is_floating(type_id) || std::any_of(array.begin(), array.end(), std::isnan)) {
       return IsMonotonicOutput(false, false, false, false, out);
     } else {
     ...
     }
   }
   ```
   [`is_floating(type_id)` is defined here](https://github.com/apache/arrow/blob/master/cpp/src/arrow/type_traits.h#L859).
   
   P.S. I did not ran this code, but something along these lines should work.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] edponce commented on a change in pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#discussion_r772815552



##########
File path: cpp/src/arrow/compute/kernels/vector_is_monotonic.cc
##########
@@ -0,0 +1,321 @@
+// 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.
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernel.h"
+#include "arrow/compute/kernels/aggregate_internal.h"
+#include "arrow/compute/registry.h"
+#include "arrow/util/optional.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+// ----------------------------------------------------------------------
+// IsMonotonic implementation
+
+using IsMonotonicState = OptionsWrapper<IsMonotonicOptions>;
+
+Status IsMonotonicOutput(bool increasing, bool strictly_increasing, bool decreasing,
+                         bool strictly_decreasing, Datum* out) {
+  ARROW_ASSIGN_OR_RAISE(
+      *out, StructScalar::Make({std::make_shared<BooleanScalar>(increasing),
+                                std::make_shared<BooleanScalar>(strictly_increasing),
+                                std::make_shared<BooleanScalar>(decreasing),
+                                std::make_shared<BooleanScalar>(strictly_decreasing)},
+                               {"increasing", "strictly_increasing", "decreasing",
+                                "strictly_decreasing"}));
+  return Status::OK();
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  // Short circuit for NaNs.
+  // https://en.wikipedia.org/wiki/NaN#Comparison_with_NaN
+  if (std::isnan(current) || std::isnan(next)) {
+    *increasing = false;
+    *strictly_increasing = false;
+    *decreasing = false;
+    *strictly_decreasing = false;
+  } else {
+    bool equal =
+        // Approximately equal within some error bound (epsilon).
+        (options.floating_approximate &&
+         (fabs(current - next) <=
+          static_cast<typename DataType::c_type>(options.epsilon))) ||
+        // Or exactly equal.
+        current == next;
+    if (*increasing) {
+      if (!(equal || next > current)) {
+        *increasing = false;
+        *strictly_increasing = false;
+      }
+    }
+    if (*decreasing) {
+      if (!(equal || next < current)) {
+        *decreasing = false;
+        *strictly_decreasing = false;
+      }
+    }
+    if (*strictly_increasing) {
+      if (equal || !(next > current)) {
+        *strictly_increasing = false;
+      }
+    }
+    if (*strictly_decreasing) {
+      if (equal || !(next < current)) {
+        *strictly_decreasing = false;
+      }
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType> IsMonotonicCheck(
+    const typename DataType::c_type& current, const typename DataType::c_type& next,
+    bool* increasing, bool* strictly_increasing, bool* decreasing,
+    bool* strictly_decreasing, const IsMonotonicOptions& options) {
+  if (*increasing) {
+    if (!(next >= current)) {
+      *increasing = false;
+      *strictly_increasing = false;
+    }
+  }
+  if (*strictly_increasing) {
+    if (!(next > current)) {
+      *strictly_increasing = false;
+    }
+  }
+  if (*decreasing) {
+    if (!(next <= current)) {
+      *decreasing = false;
+      *strictly_decreasing = false;
+    }
+  }
+  if (*strictly_decreasing) {
+    if (!(next < current)) {
+      *strictly_decreasing = false;
+    }
+  }
+}
+
+template <typename DataType>
+enable_if_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return opt.has_value() && std::isnan(opt.value());
+}
+
+template <typename DataType>
+enable_if_not_floating_point<DataType, bool> isnan(
+    const util::optional<typename DataType::c_type>& opt) {
+  return false;

Review comment:
       My comment below suggests a change that would allow you to get rid of the `enable_if_not_floating_point<...> isnan()` variant.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] mbrobbel edited a comment on pull request #11937: ARROW-1569: [C++] Kernel functions for determining monotonicity (ascending or descending) for well-ordered types

Posted by GitBox <gi...@apache.org>.
mbrobbel edited a comment on pull request #11937:
URL: https://github.com/apache/arrow/pull/11937#issuecomment-996683557


   >     * `IsMonotonic` needs to be consistent with corresponding [sorting functions](https://arrow.apache.org/docs/cpp/compute.html#sorts-and-partitions), such that `IsMonotonic(input) == (Sort(input) == input)`.
   
   I agree.
   
   >     * Currently `IsMonotonic` outputs a struct describing the monotonic properties of the data. What your thoughts on having a convenience wrapper function that receives `FunctionOptions` with a single requested monotonic behavior? For example, `IsMonotonic(input, MonotonicOptions.StrictlyIncreasing)`. This would provide a more readable API for client code and bypass the need to unpack the `StructScalar` output to check the corresponding monotonic behavior of interest.
   
   I initially set it up like that but @bkietz suggested to output a struct scalar instead (like the min/max kernel).
   
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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