You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by fs...@apache.org on 2019/08/02 18:51:00 UTC

[arrow] branch master updated: ARROW-1566: [C++] Implement non-materializing sort kernels

This is an automated email from the ASF dual-hosted git repository.

fsaintjacques pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 0e6078c  ARROW-1566: [C++] Implement non-materializing sort kernels
0e6078c is described below

commit 0e6078c771163306df745e64e07d8bd56a459aca
Author: Artem Alekseev <ar...@intel.com>
AuthorDate: Fri Aug 2 14:50:43 2019 -0400

    ARROW-1566: [C++] Implement non-materializing sort kernels
    
    Initial implementation of the requested kernel.
    
    Closes #4861 from fexolm/sort-kernel and squashes the following commits:
    
    390bccc62 <Artem Alekseev> Merge branch 'master' into sort-kernel
    089b472c4 <Artem Alekseev> trigger CI
    3ab238e8a <Artem Alekseev> Merge remote-tracking branch 'upstream/master' into sort-kernel
    d0a83f10b <Artem Alekseev> retrigger CI
    5f008c29f <Artem Alekseev> Trigger CI
    d2b272906 <Artem Alekseev> merge conflicts
    bded09c46 <Artem Alekseev> minor changes
    a4511b088 <Artem Alekseev> trigger CI
    5fc5d1ca1 <Artem Alekseev> fix review comments
    67be764f6 <Artem Alekseev> fix compiler warnings
    71fcfb2d1 <Artem Alekseev> fix linter warnings
    c3d0b4db3 <Artem Alekseev> add tests and benchmarks
    c0b5955b2 <Artem Alekseev> use null-aware stable sort
    29998a085 <Artem Alekseev> cleanup tests
    77a560be5 <Artem Alekseev>  Implement Sort kernel
    
    Lead-authored-by: Artem Alekseev <ar...@intel.com>
    Co-authored-by: Artem Alekseev <14...@users.noreply.github.com>
    Signed-off-by: François Saint-Jacques <fs...@gmail.com>
---
 cpp/src/arrow/CMakeLists.txt                       |   1 +
 cpp/src/arrow/compute/kernels/CMakeLists.txt       |   2 +
 cpp/src/arrow/compute/kernels/argsort-benchmark.cc |  59 +++++++
 cpp/src/arrow/compute/kernels/argsort-test.cc      | 191 +++++++++++++++++++++
 cpp/src/arrow/compute/kernels/argsort.cc           | 190 ++++++++++++++++++++
 cpp/src/arrow/compute/kernels/argsort.h            |  51 ++++++
 cpp/src/arrow/testing/gtest_util.h                 |   6 +
 7 files changed, 500 insertions(+)

diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index e5820b2..7d385e0 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -188,6 +188,7 @@ if(ARROW_COMPUTE)
       compute/kernels/take.cc
       compute/kernels/isin.cc
       compute/kernels/util-internal.cc
+      compute/kernels/argsort.cc
       compute/operations/cast.cc
       compute/operations/literal.cc)
 endif()
diff --git a/cpp/src/arrow/compute/kernels/CMakeLists.txt b/cpp/src/arrow/compute/kernels/CMakeLists.txt
index 2702a73..529cb50 100644
--- a/cpp/src/arrow/compute/kernels/CMakeLists.txt
+++ b/cpp/src/arrow/compute/kernels/CMakeLists.txt
@@ -20,8 +20,10 @@ arrow_install_all_headers("arrow/compute/kernels")
 add_arrow_test(boolean-test PREFIX "arrow-compute")
 add_arrow_test(cast-test PREFIX "arrow-compute")
 add_arrow_test(hash-test PREFIX "arrow-compute")
+add_arrow_test(argsort-test PREFIX "arrow-compute")
 add_arrow_test(isin-test PREFIX "arrow-compute")
 add_arrow_test(util-internal-test PREFIX "arrow-compute")
+add_arrow_benchmark(argsort-benchmark PREFIX "arrow-compute")
 
 # Aggregates
 add_arrow_test(aggregate-test PREFIX "arrow-compute")
diff --git a/cpp/src/arrow/compute/kernels/argsort-benchmark.cc b/cpp/src/arrow/compute/kernels/argsort-benchmark.cc
new file mode 100644
index 0000000..ef0ebb5
--- /dev/null
+++ b/cpp/src/arrow/compute/kernels/argsort-benchmark.cc
@@ -0,0 +1,59 @@
+// 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 "benchmark/benchmark.h"
+
+#include "arrow/compute/kernels/argsort.h"
+
+#include "arrow/compute/benchmark-util.h"
+#include "arrow/compute/test-util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+
+namespace arrow {
+namespace compute {
+constexpr auto kSeed = 0x0ff1ce;
+
+static void ArgsortBenchmark(benchmark::State& state,
+                             const std::shared_ptr<Array>& values) {
+  FunctionContext ctx;
+  for (auto _ : state) {
+    std::shared_ptr<Array> out;
+    ABORT_NOT_OK(Argsort(&ctx, *values, &out));
+    benchmark::DoNotOptimize(out);
+  }
+}
+
+static void ArgsortInt64(benchmark::State& state) {
+  RegressionArgs args(state);
+
+  const int64_t array_size = args.size / sizeof(int64_t);
+  auto rand = random::RandomArrayGenerator(kSeed);
+
+  auto values = rand.Int64(array_size, -100, 100, args.null_proportion);
+
+  ArgsortBenchmark(state, values);
+}
+
+BENCHMARK(ArgsortInt64)
+    ->Apply(RegressionSetArgs)
+    ->Args({1 << 20, 1})
+    ->Args({1 << 23, 1})
+    ->MinTime(1.0)
+    ->Unit(benchmark::TimeUnit::kNanosecond);
+}  // namespace compute
+}  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/argsort-test.cc b/cpp/src/arrow/compute/kernels/argsort-test.cc
new file mode 100644
index 0000000..d879139
--- /dev/null
+++ b/cpp/src/arrow/compute/kernels/argsort-test.cc
@@ -0,0 +1,191 @@
+// 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 <memory>
+#include <vector>
+
+#include "arrow/compute/context.h"
+#include "arrow/compute/kernels/argsort.h"
+#include "arrow/compute/test-util.h"
+#include "arrow/testing/gtest_common.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/testing/util.h"
+#include "arrow/type_traits.h"
+
+namespace arrow {
+namespace compute {
+
+template <typename ArrowType>
+class TestArgsortKernel : public ComputeFixture, public TestBase {
+ private:
+  void AssertArgsortArrays(const std::shared_ptr<Array> values,
+                           const std::shared_ptr<Array> expected) {
+    std::shared_ptr<Array> actual;
+    ASSERT_OK(arrow::compute::Argsort(&this->ctx_, *values, &actual));
+    ASSERT_OK(ValidateArray(*actual));
+    AssertArraysEqual(*expected, *actual);
+  }
+
+ protected:
+  virtual void AssertArgsort(const std::string& values, const std::string& expected) {
+    auto type = TypeTraits<ArrowType>::type_singleton();
+    AssertArgsortArrays(ArrayFromJSON(type, values), ArrayFromJSON(uint64(), expected));
+  }
+};
+
+template <typename ArrowType>
+class TestArgsortKernelForReal : public TestArgsortKernel<ArrowType> {};
+TYPED_TEST_CASE(TestArgsortKernelForReal, RealArrowTypes);
+
+template <typename ArrowType>
+class TestArgsortKernelForIntegral : public TestArgsortKernel<ArrowType> {};
+TYPED_TEST_CASE(TestArgsortKernelForIntegral, IntegralArrowTypes);
+
+template <typename ArrowType>
+class TestArgsortKernelForStrings : public TestArgsortKernel<ArrowType> {};
+TYPED_TEST_CASE(TestArgsortKernelForStrings, testing::Types<StringType>);
+
+TYPED_TEST(TestArgsortKernelForReal, SortReal) {
+  this->AssertArgsort("[]", "[]");
+
+  this->AssertArgsort("[3.4, 2.6, 6.3]", "[1, 0, 2]");
+
+  this->AssertArgsort("[1.1, 2.4, 3.5, 4.3, 5.1, 6.8, 7.3]", "[0,1,2,3,4,5,6]");
+
+  this->AssertArgsort("[7, 6, 5, 4, 3, 2, 1]", "[6,5,4,3,2,1,0]");
+
+  this->AssertArgsort("[10.4, 12, 4.2, 50, 50.3, 32, 11]", "[2,0,6,1,5,3,4]");
+
+  this->AssertArgsort("[null, 1, 3.3, null, 2, 5.3]", "[1,4,2,5,0,3]");
+}
+
+TYPED_TEST(TestArgsortKernelForIntegral, SortIntegral) {
+  this->AssertArgsort("[]", "[]");
+
+  this->AssertArgsort("[3, 2, 6]", "[1, 0, 2]");
+
+  this->AssertArgsort("[1, 2, 3, 4, 5, 6, 7]", "[0,1,2,3,4,5,6]");
+
+  this->AssertArgsort("[7, 6, 5, 4, 3, 2, 1]", "[6,5,4,3,2,1,0]");
+
+  this->AssertArgsort("[10, 12, 4, 50, 50, 32, 11]", "[2,0,6,1,5,3,4]");
+
+  this->AssertArgsort("[null, 1, 3, null, 2, 5]", "[1,4,2,5,0,3]");
+}
+
+TYPED_TEST(TestArgsortKernelForStrings, SortStrings) {
+  this->AssertArgsort("[]", "[]");
+
+  this->AssertArgsort(R"(["a", "b", "c"])", "[0, 1, 2]");
+
+  this->AssertArgsort(R"(["foo", "bar", "baz"])", "[1,2,0]");
+
+  this->AssertArgsort(R"(["testing", "sort", "for", "strings"])", "[2, 1, 3, 0]");
+}
+
+template <typename ArrowType>
+class TestArgsortKernelRandom : public ComputeFixture, public TestBase {};
+
+using ArgsortableTypes =
+    ::testing::Types<UInt8Type, UInt16Type, UInt32Type, UInt64Type, Int8Type, Int16Type,
+                     Int32Type, Int64Type, FloatType, DoubleType, StringType>;
+
+template <typename ArrayType>
+class Comparator {
+ public:
+  bool operator()(const ArrayType& array, uint64_t lhs, uint64_t rhs) {
+    if (array.IsNull(rhs) && array.IsNull(lhs)) return lhs < rhs;
+    if (array.IsNull(rhs)) return true;
+    if (array.IsNull(lhs)) return false;
+    return array.Value(lhs) <= array.Value(rhs);
+  }
+};
+
+template <>
+class Comparator<StringArray> {
+ public:
+  bool operator()(const BinaryArray& array, uint64_t lhs, uint64_t rhs) {
+    if (array.IsNull(rhs) && array.IsNull(lhs)) return lhs < rhs;
+    if (array.IsNull(rhs)) return true;
+    if (array.IsNull(lhs)) return false;
+    return array.GetView(lhs) <= array.GetView(rhs);
+  }
+};
+
+template <typename ArrayType>
+void ValidateSorted(const ArrayType& array, UInt64Array& offsets) {
+  Comparator<ArrayType> compare;
+  for (int i = 1; i < array.length(); i++) {
+    uint64_t lhs = offsets.Value(i - 1);
+    uint64_t rhs = offsets.Value(i);
+    ASSERT_TRUE(compare(array, lhs, rhs));
+  }
+}
+
+class RandomImpl {
+ protected:
+  random::RandomArrayGenerator generator;
+
+ public:
+  explicit RandomImpl(random::SeedType seed) : generator(seed) {}
+};
+
+template <typename ArrowType>
+class Random : public RandomImpl {
+  using CType = typename TypeTraits<ArrowType>::CType;
+
+ public:
+  explicit Random(random::SeedType seed) : RandomImpl(seed) {}
+
+  std::shared_ptr<Array> Generate(uint64_t count, double null_prob) {
+    return generator.Numeric<ArrowType>(count, std::numeric_limits<CType>::min(),
+                                        std::numeric_limits<CType>::max(), null_prob);
+  }
+};
+
+template <>
+class Random<StringType> : public RandomImpl {
+ public:
+  explicit Random(random::SeedType seed) : RandomImpl(seed) {}
+
+  std::shared_ptr<Array> Generate(uint64_t count, double null_prob) {
+    return generator.String(count, 1, 100, null_prob);
+  }
+};
+
+TYPED_TEST_CASE(TestArgsortKernelRandom, ArgsortableTypes);
+
+TYPED_TEST(TestArgsortKernelRandom, SortRandomValues) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+
+  Random<TypeParam> rand(0x5487655);
+  int times = 5;
+  int length = 10000;
+  for (int test = 0; test < times; test++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      auto array = rand.Generate(length, null_probability);
+      std::shared_ptr<Array> offsets;
+      ASSERT_OK(arrow::compute::Argsort(&this->ctx_, *array, &offsets));
+      ValidateSorted<ArrayType>(*std::static_pointer_cast<ArrayType>(array),
+                                *std::static_pointer_cast<UInt64Array>(offsets));
+    }
+  }
+}
+
+}  // namespace compute
+}  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/argsort.cc b/cpp/src/arrow/compute/kernels/argsort.cc
new file mode 100644
index 0000000..c41ded6
--- /dev/null
+++ b/cpp/src/arrow/compute/kernels/argsort.cc
@@ -0,0 +1,190 @@
+// 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/kernels/argsort.h"
+#include <algorithm>
+#include <numeric>
+#include <vector>
+#include "arrow/builder.h"
+#include "arrow/compute/context.h"
+#include "arrow/compute/expression.h"
+#include "arrow/compute/logical_type.h"
+#include "arrow/type_traits.h"
+namespace arrow {
+
+class Array;
+
+namespace compute {
+
+/// \brief UnaryKernel implementing Argsort operation
+class ARROW_EXPORT ArgsortKernel : public UnaryKernel {
+ protected:
+  std::shared_ptr<DataType> type_;
+
+ public:
+  /// \brief UnaryKernel interface
+  ///
+  /// delegates to subclasses via Argsort()
+  Status Call(FunctionContext* ctx, const Datum& values, Datum* offsets) override = 0;
+
+  /// \brief output type of this kernel
+  std::shared_ptr<DataType> out_type() const override { return uint64(); }
+
+  /// \brief single-array implementation
+  virtual Status Argsort(FunctionContext* ctx, const std::shared_ptr<Array>& values,
+                         std::shared_ptr<Array>* offsets) = 0;
+
+  /// \brief factory for ArgsortKernel
+  ///
+  /// \param[in] value_type constructed ArgsortKernel will support sorting
+  ///            values of this type
+  /// \param[out] out created kernel
+  static Status Make(const std::shared_ptr<DataType>& value_type,
+                     std::unique_ptr<ArgsortKernel>* out);
+};
+
+template <typename ArrayType>
+bool CompareValues(const ArrayType& array, uint64_t lhs, uint64_t rhs) {
+  return array.Value(lhs) < array.Value(rhs);
+}
+
+template <typename ArrayType>
+bool CompareViews(const ArrayType& array, uint64_t lhs, uint64_t rhs) {
+  return array.GetView(lhs) < array.GetView(rhs);
+}
+
+template <typename ArrowType, typename Comparator>
+class ArgsortKernelImpl : public ArgsortKernel {
+  using ArrayType = typename TypeTraits<ArrowType>::ArrayType;
+
+ public:
+  explicit ArgsortKernelImpl(Comparator compare) : compare_(compare) {}
+
+  Status Argsort(FunctionContext* ctx, const std::shared_ptr<Array>& values,
+                 std::shared_ptr<Array>* offsets) {
+    return ArgsortImpl(ctx, std::static_pointer_cast<ArrayType>(values), offsets);
+  }
+
+  Status Call(FunctionContext* ctx, const Datum& values, Datum* offsets) {
+    if (!values.is_array()) {
+      return Status::Invalid("ArgsortKernel expects array values");
+    }
+    auto values_array = values.make_array();
+    std::shared_ptr<Array> offsets_array;
+    RETURN_NOT_OK(this->Argsort(ctx, values_array, &offsets_array));
+    *offsets = offsets_array;
+    return Status::OK();
+  }
+
+  std::shared_ptr<DataType> out_type() const { return type_; }
+
+ private:
+  Comparator compare_;
+
+  Status ArgsortImpl(FunctionContext* ctx, const std::shared_ptr<ArrayType>& values,
+                     std::shared_ptr<Array>* offsets) {
+    std::shared_ptr<Buffer> indices_buf;
+    int64_t buf_size = values->length() * sizeof(uint64_t);
+    RETURN_NOT_OK(AllocateBuffer(ctx->memory_pool(), buf_size, &indices_buf));
+
+    int64_t* indices_begin = reinterpret_cast<int64_t*>(indices_buf->mutable_data());
+    int64_t* indices_end = indices_begin + values->length();
+
+    std::iota(indices_begin, indices_end, 0);
+    auto nulls_begin = indices_end;
+    if (values->null_count()) {
+      nulls_begin =
+          std::stable_partition(indices_begin, indices_end,
+                                [&values](uint64_t ind) { return !values->IsNull(ind); });
+    }
+    std::stable_sort(indices_begin, nulls_begin,
+                     [&values, this](uint64_t left, uint64_t right) {
+                       return compare_(*values, left, right);
+                     });
+    *offsets = std::make_shared<UInt64Array>(values->length(), indices_buf);
+    return Status::OK();
+  }
+};
+
+template <typename ArrowType, typename Comparator>
+ArgsortKernelImpl<ArrowType, Comparator>* MakeArgsortKernelImpl(Comparator comparator) {
+  return new ArgsortKernelImpl<ArrowType, Comparator>(comparator);
+}
+
+Status ArgsortKernel::Make(const std::shared_ptr<DataType>& value_type,
+                           std::unique_ptr<ArgsortKernel>* out) {
+  ArgsortKernel* kernel;
+  switch (value_type->id()) {
+    case Type::UINT8:
+      kernel = MakeArgsortKernelImpl<UInt8Type>(CompareValues<UInt8Array>);
+      break;
+    case Type::INT8:
+      kernel = MakeArgsortKernelImpl<Int8Type>(CompareValues<Int8Array>);
+      break;
+    case Type::UINT16:
+      kernel = MakeArgsortKernelImpl<UInt16Type>(CompareValues<UInt16Array>);
+      break;
+    case Type::INT16:
+      kernel = MakeArgsortKernelImpl<Int16Type>(CompareValues<Int16Array>);
+      break;
+    case Type::UINT32:
+      kernel = MakeArgsortKernelImpl<UInt32Type>(CompareValues<UInt32Array>);
+      break;
+    case Type::INT32:
+      kernel = MakeArgsortKernelImpl<Int32Type>(CompareValues<Int32Array>);
+      break;
+    case Type::UINT64:
+      kernel = MakeArgsortKernelImpl<UInt64Type>(CompareValues<UInt64Array>);
+      break;
+    case Type::INT64:
+      kernel = MakeArgsortKernelImpl<Int64Type>(CompareValues<Int64Array>);
+      break;
+    case Type::FLOAT:
+      kernel = MakeArgsortKernelImpl<FloatType>(CompareValues<FloatArray>);
+      break;
+    case Type::DOUBLE:
+      kernel = MakeArgsortKernelImpl<DoubleType>(CompareValues<DoubleArray>);
+      break;
+    case Type::BINARY:
+      kernel = MakeArgsortKernelImpl<BinaryType>(CompareViews<BinaryArray>);
+      break;
+    case Type::STRING:
+      kernel = MakeArgsortKernelImpl<StringType>(CompareViews<StringArray>);
+      break;
+    default:
+      return Status::NotImplemented("Sorting of ", *value_type, " arrays");
+  }
+  out->reset(kernel);
+  return Status::OK();
+}
+
+Status Argsort(FunctionContext* ctx, const Datum& values, Datum* offsets) {
+  std::unique_ptr<ArgsortKernel> kernel;
+  RETURN_NOT_OK(ArgsortKernel::Make(values.type(), &kernel));
+  return kernel->Call(ctx, values, offsets);
+}
+
+Status Argsort(FunctionContext* ctx, const Array& values,
+               std::shared_ptr<Array>* offsets) {
+  Datum offsets_datum;
+  RETURN_NOT_OK(Argsort(ctx, Datum(values.data()), &offsets_datum));
+  *offsets = offsets_datum.make_array();
+  return Status::OK();
+}
+
+}  // namespace compute
+}  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/argsort.h b/cpp/src/arrow/compute/kernels/argsort.h
new file mode 100644
index 0000000..f2d106d
--- /dev/null
+++ b/cpp/src/arrow/compute/kernels/argsort.h
@@ -0,0 +1,51 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+
+#include "arrow/compute/kernel.h"
+#include "arrow/status.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class Array;
+
+namespace compute {
+
+class FunctionContext;
+
+/// \brief Returns the indices that would sort an array.
+///
+/// Perform an indirect sort of array. The output array will contain
+/// indices that would sort an array, which would be the same length
+/// as input. Nulls will be stably partitioned to the end of the output.
+///
+/// For example given values = [null, 1, 3.3, null, 2, 5.3], the output
+/// will be [1, 4, 2, 5, 0, 3]
+///
+/// \param[in] ctx the FunctionContext
+/// \param[in] values array to sort
+/// \param[out] offsets indices that would sort an array
+ARROW_EXPORT
+Status Argsort(FunctionContext* ctx, const Array& values,
+               std::shared_ptr<Array>* offsets);
+
+}  // namespace compute
+}  // namespace arrow
diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h
index e7c4bb3..f378b80 100644
--- a/cpp/src/arrow/testing/gtest_util.h
+++ b/cpp/src/arrow/testing/gtest_util.h
@@ -118,6 +118,12 @@ typedef ::testing::Types<UInt8Type, UInt16Type, UInt32Type, UInt64Type, Int8Type
                          Int16Type, Int32Type, Int64Type, FloatType, DoubleType>
     NumericArrowTypes;
 
+typedef ::testing::Types<FloatType, DoubleType> RealArrowTypes;
+
+typedef testing::Types<UInt8Type, UInt16Type, UInt32Type, UInt64Type, Int8Type, Int16Type,
+                       Int32Type, Int64Type>
+    IntegralArrowTypes;
+
 class Array;
 class ChunkedArray;
 class RecordBatch;