You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "felipecrv (via GitHub)" <gi...@apache.org> on 2023/02/15 12:55:32 UTC

[GitHub] [arrow] felipecrv opened a new pull request, #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

felipecrv opened a new pull request, #34195:
URL: https://github.com/apache/arrow/pull/34195

   Also closes #20351.
   
   Currently this includes commits from #33641, but I will rebase once that is merged and then this PR can be easily reviewed.


-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1119476602


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,652 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+

Review Comment:
   The function definition currently declares kernels for `numeric`, `bool`, and `null` types. The `Type::NA` implementation was a TODO in the initial code by Tobias, and I added a specialized implementation for them. I think I will need something similar that doesn't use this for fixed-length binary arrays. Can I create a separate issue for those and work on getting this PR merged with these initial types? I'm trying to get to a steady flow of small PRs.



-- 
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] felipecrv commented on pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1446968372

   @westonpace @zeroshade @lidavidm this is now ready for review.


-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1126923075


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));
+    ARROW_CHECK_OK(builder->AppendArraySlice(ArraySpan(*child), 0, child->length));
+    array->child_data[1] = builder->Finish().ValueOrDie()->Slice(offset)->data();
+  }
+};
+
+TEST_P(TestRunEndEncodeDecode, EncodeDecodeArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  auto run_ends_array = MakeArray(encoded->child_data[0]);
+  auto values_array = MakeArray(encoded->child_data[1]);
+  ASSERT_OK(MakeArray(encoded)->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*ArrayFromJSON(run_ends_type, data.expected_run_ends_json),
+                      *run_ends_array);
+  ASSERT_ARRAYS_EQUAL(*values_array, *data.expected_values);
+  ASSERT_EQ(encoded->buffers.size(), 1);
+  ASSERT_EQ(encoded->buffers[0], NULLPTR);
+  ASSERT_EQ(encoded->child_data.size(), 2);
+  ASSERT_EQ(run_ends_array->data()->buffers[0], NULLPTR);
+  ASSERT_EQ(run_ends_array->length(), data.expected_values->length());
+  ASSERT_EQ(run_ends_array->offset(), 0);
+  ASSERT_EQ(encoded->length, data.input->length());
+  ASSERT_EQ(encoded->offset, 0);
+  ASSERT_EQ(*encoded->type, RunEndEncodedType(run_ends_type, data.input->type()));
+  ASSERT_EQ(encoded->null_count, 0);
+
+  ASSERT_OK_AND_ASSIGN(Datum decoded_datum, RunEndDecode(encoded));
+  auto decoded = decoded_datum.make_array();
+  ASSERT_OK(decoded->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*decoded, *data.input);
+}
+
+// Encoding an input with an offset results in a completely new encoded array without an
+// offset. This means The EncodeDecodeArray test will never actually decode an array
+// with an offset, even though we have inputs with offsets. This test slices one element
+// off the encoded array and decodes that.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffset) {
+  auto [data, run_ends_type] = GetParam();
+  if (data.input->length() == 0) {
+    return;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first,
+                       RunEndDecode(encoded->Slice(1, encoded->length - 1)));
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_last,
+                       RunEndDecode(encoded->Slice(0, encoded->length - 1)));
+  auto array_without_first = datum_without_first.make_array();
+  auto array_without_last = datum_without_last.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_OK(array_without_last->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input->Slice(1));
+  ASSERT_ARRAYS_EQUAL(*array_without_last,
+                      *data.input->Slice(0, data.input->length() - 1));
+}
+
+// This test creates an run-end encoded array with an offset in the child array, which
+// removes the first run in the test data.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffsetInChildArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  this->AddArtificialOffsetInChildArray(encoded.get(), 100);
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first, RunEndDecode(encoded));
+  auto array_without_first = datum_without_first.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input);
+}
+
+INSTANTIATE_TEST_SUITE_P(

Review Comment:
   The implementation doesn't handle `RunEndEncodedScalar`, so I will have to add that first ;-) Thanks for reminding me of that.



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

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 #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1447070712

   :warning: GitHub issue #32105 **has been automatically assigned in GitHub** to PR creator.


-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1127036928


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;

Review Comment:
   Pushed now.



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));

Review Comment:
   This is also pushed now.



-- 
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] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130212460


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;

Review Comment:
   I'll note that that header predates the C++17 migration, so if you think `if constexpr` can clean up the code over template metaprogramming, I'd lean towards that



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

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

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


[GitHub] [arrow] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1119476602


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,652 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+

Review Comment:
   The function definition currently declares kernels for `numeric`, `bool`, and `null` types. The `Type::NA` implementation was a TODO in the initial code by Tobias, and I added a specialized implementation for them. I think I will need something similar that doesn't use this for fixed-length binary arrays. Can I create a separate issue for those and work on getting this PR merged with these initial types. I'm trying to get to a steady flow of small PRs.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130213285


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }

Review Comment:
   Indeed, and having this `cmov` waiting for the `valid` to be loaded is bad for speculative execution. I will fix.



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1132839717


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),
+                                           ctx_->memory_pool()));
+
+      auto ree_type = std::make_shared<RunEndEncodedType>(
+          std::make_shared<RunEndType>(), input_array_.type->GetSharedPtr());
+      auto run_ends_data =
+          ArrayData::Make(ree_type->run_end_type(), num_output_runs,
+                          {NULLPTR, std::move(run_ends_buffer)}, /*null_count=*/0);
+      auto values_data =
+          ArrayData::Make(ree_type->value_type(), num_output_runs,
+                          {std::move(validity_buffer), std::move(values_buffer)},
+                          /*null_count=*/num_output_runs - num_valid_runs);
+
+      output_array_data =
+          ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                          {std::move(run_ends_data), std::move(values_data)},
+                          /*null_count=*/0);
+    }
+
+    if (input_length > 0) {
+      // Initialize the output pointers
+      auto* output_run_ends =
+          output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+      auto* output_validity =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(0);
+      auto* output_values =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(1);
+
+      if constexpr (has_validity_buffer) {
+        // Clear last byte in validity buffer to ensure padding bits are zeroed
+        output_validity[validity_buffer_size - 1] = 0;
+      }
+
+      // Second pass: write the runs
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> writing_loop(
+          input_length, input_offset, input_validity, input_values, output_validity,
+          output_values, output_run_ends);
+      [[maybe_unused]] int64_t num_written_runs = writing_loop.WriteEncodedRuns();
+      DCHECK_EQ(num_written_runs, num_output_runs);
+    }
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+template <typename RunEndType>
+Status RunEndEncodeNullArray(KernelContext* ctx, const ArraySpan& input_array,
+                             ExecResult* output) {
+  using RunEndCType = typename RunEndType::c_type;
+
+  const int64_t input_length = input_array.length;
+  auto input_array_type = input_array.type->GetSharedPtr();
+  DCHECK(input_array_type->id() == Type::NA);
+
+  int64_t num_output_runs = 0;
+  if (input_length > 0) {
+    // Abort if run-end type cannot hold the input length
+    RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_array.length));
+    num_output_runs = 1;
+  }
+
+  // Allocate the output array data
+  std::shared_ptr<ArrayData> output_array_data;
+  {
+    ARROW_ASSIGN_OR_RAISE(
+        auto run_ends_buffer,
+        AllocateBuffer(num_output_runs * RunEndType().bit_width(), ctx->memory_pool()));
+
+    auto ree_type = std::make_shared<RunEndEncodedType>(std::make_shared<RunEndType>(),
+                                                        input_array_type);
+    auto run_ends_data = ArrayData::Make(std::make_shared<RunEndType>(), num_output_runs,
+                                         {NULLPTR, std::move(run_ends_buffer)},
+                                         /*null_count=*/0);
+    auto values_data = ArrayData::Make(input_array_type, num_output_runs, {NULLPTR},
+                                       /*null_count=*/num_output_runs);
+
+    output_array_data =
+        ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                        {std::move(run_ends_data), std::move(values_data)},
+                        /*null_count=*/0);
+  }

Review Comment:
   okay that's fair in this case.



-- 
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] ursabot commented on pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "ursabot (via GitHub)" <gi...@apache.org>.
ursabot commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1467136355

   Benchmark runs are scheduled for baseline = afa362e5a0b6cb9a48ce8635057f1c721d9aa244 and contender = dbc1e90b58daa271cfdadffdaac957072e07cf82. dbc1e90b58daa271cfdadffdaac957072e07cf82 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/252584a1912c40d88d37534616a0b80f...9adc08b0d7464ce79d4f737e83fd7429/)
   [Failed :arrow_down:0.18% :arrow_up:0.3%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/73dea22bdee845238fa17f2aacbbf548...121b62d360e84c9fa8f82231e32f03a2/)
   [Finished :arrow_down:0.77% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/2335b2ca82694c7d9ec25b7fe03687fa...9b119428ff314308ac078f95f3db32e2/)
   [Finished :arrow_down:0.69% :arrow_up:0.16%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/044a0986beb14e72b606ac43866d6744...98d33342eb4242cabbf18a345db2e7d8/)
   Buildkite builds:
   [Finished] [`dbc1e90b` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2515)
   [Finished] [`dbc1e90b` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2545)
   [Finished] [`dbc1e90b` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2513)
   [Finished] [`dbc1e90b` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2536)
   [Finished] [`afa362e5` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2514)
   [Failed] [`afa362e5` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2544)
   [Finished] [`afa362e5` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2512)
   [Finished] [`afa362e5` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2535)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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 #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1431325419

   :warning: GitHub issue #32105 **has been automatically assigned in GitHub** to PR creator.


-- 
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] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1127031432


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {

Review Comment:
   Just a suggestion - if it wouldn't improve it (and it doesn't seem like it could be a variant of one of the visitors) then don't worry about 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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1127037813


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {

Review Comment:
   Can you take a look at the last commit? The implementation is longer and relies on a lot of inlining to be efficient.



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1119382712


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,652 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+

Review Comment:
   Shouldn't we need another explicit implementation here for handling variable and fixed length binary arrays? I don't see anything elsewhere for handling encoding/decoding of that type.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130215628


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {

Review Comment:
   I tried that at first, the code got very repetitive and more bug-prone. I would have to repeat the next block with a hard-coded 0.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130229041


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;

Review Comment:
   I rewrote it to use the `if constexpr (is_same_type<...>)` and I will wait to know what you think about following the pattern in `visit_data_inline.h`.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1120135574


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,652 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+

Review Comment:
   Oh, right. Added a TODO mentioned the newly created issue.



-- 
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 #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1444977638

   :warning: GitHub issue #32105 **has been automatically assigned in GitHub** to PR creator.


-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1132842018


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   Good news. I just added a commit that removes the need for `BytesForBits` and does the right clearing of trailing bits when allocating bitmaps.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1126908450


##########
cpp/src/arrow/compute/api_vector.h:
##########
@@ -577,6 +586,30 @@ Result<Datum> DictionaryEncode(
     const DictionaryEncodeOptions& options = DictionaryEncodeOptions::Defaults(),
     ExecContext* ctx = NULLPTR);
 
+/// \brief Run-end-encode values in an array-like object
+///
+/// \param[in] value array-like input
+/// \param[in] options configures encoding behavior
+/// \param[in] ctx the function execution context, optional
+/// \return result with same shape and type as input

Review Comment:
   Fixed. Pushing soon.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1131321539


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   I opened a PR adding `BytesForBits` in the filtering code after this research: https://github.com/apache/arrow/pull/34521



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1131351137


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;

Review Comment:
   I just opened #34522 which removes the BooleanType specialization from `visit_data_inline.h`, but leaves that `enable_if` for selecting implementations across different categories of types.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1128719263


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));
+    ARROW_CHECK_OK(builder->AppendArraySlice(ArraySpan(*child), 0, child->length));
+    array->child_data[1] = builder->Finish().ValueOrDie()->Slice(offset)->data();
+  }
+};
+
+TEST_P(TestRunEndEncodeDecode, EncodeDecodeArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  auto run_ends_array = MakeArray(encoded->child_data[0]);
+  auto values_array = MakeArray(encoded->child_data[1]);
+  ASSERT_OK(MakeArray(encoded)->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*ArrayFromJSON(run_ends_type, data.expected_run_ends_json),
+                      *run_ends_array);
+  ASSERT_ARRAYS_EQUAL(*values_array, *data.expected_values);
+  ASSERT_EQ(encoded->buffers.size(), 1);
+  ASSERT_EQ(encoded->buffers[0], NULLPTR);
+  ASSERT_EQ(encoded->child_data.size(), 2);
+  ASSERT_EQ(run_ends_array->data()->buffers[0], NULLPTR);
+  ASSERT_EQ(run_ends_array->length(), data.expected_values->length());
+  ASSERT_EQ(run_ends_array->offset(), 0);
+  ASSERT_EQ(encoded->length, data.input->length());
+  ASSERT_EQ(encoded->offset, 0);
+  ASSERT_EQ(*encoded->type, RunEndEncodedType(run_ends_type, data.input->type()));
+  ASSERT_EQ(encoded->null_count, 0);
+
+  ASSERT_OK_AND_ASSIGN(Datum decoded_datum, RunEndDecode(encoded));
+  auto decoded = decoded_datum.make_array();
+  ASSERT_OK(decoded->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*decoded, *data.input);
+}
+
+// Encoding an input with an offset results in a completely new encoded array without an
+// offset. This means The EncodeDecodeArray test will never actually decode an array
+// with an offset, even though we have inputs with offsets. This test slices one element
+// off the encoded array and decodes that.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffset) {
+  auto [data, run_ends_type] = GetParam();
+  if (data.input->length() == 0) {
+    return;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first,
+                       RunEndDecode(encoded->Slice(1, encoded->length - 1)));
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_last,
+                       RunEndDecode(encoded->Slice(0, encoded->length - 1)));
+  auto array_without_first = datum_without_first.make_array();
+  auto array_without_last = datum_without_last.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_OK(array_without_last->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input->Slice(1));
+  ASSERT_ARRAYS_EQUAL(*array_without_last,
+                      *data.input->Slice(0, data.input->length() - 1));
+}
+
+// This test creates an run-end encoded array with an offset in the child array, which
+// removes the first run in the test data.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffsetInChildArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  this->AddArtificialOffsetInChildArray(encoded.get(), 100);
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first, RunEndDecode(encoded));
+  auto array_without_first = datum_without_first.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input);
+}
+
+INSTANTIATE_TEST_SUITE_P(

Review Comment:
   Done. Lots of chunked array tests!



-- 
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 #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1431325364

   * Closes: #32105


-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1131282326


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   `Boolean::bit_width()` returns 1:
   
   
   ```cpp
   // BooleanType within arrow use a single bit instead of the C 8-bits layout.
   int bit_width() const final { return 1; }
   ```
   
   Then the `BooleanType::byte_width()` which is the default from `DataType`
   
   ```cpp
       /// \brief Returns the type's fixed byte width, if any. Returns -1
       /// for non-fixed-width types, and should only be used for
       /// subclasses of FixedWidthType
       virtual int32_t byte_width() const final {
         int32_t num_bits = this->bit_width();
         return num_bits > 0 ? num_bits / 8 : -1;
       }
     ```
   
   evaluates to 0.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130219398


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),
+                                           ctx_->memory_pool()));
+
+      auto ree_type = std::make_shared<RunEndEncodedType>(
+          std::make_shared<RunEndType>(), input_array_.type->GetSharedPtr());
+      auto run_ends_data =
+          ArrayData::Make(ree_type->run_end_type(), num_output_runs,
+                          {NULLPTR, std::move(run_ends_buffer)}, /*null_count=*/0);
+      auto values_data =
+          ArrayData::Make(ree_type->value_type(), num_output_runs,
+                          {std::move(validity_buffer), std::move(values_buffer)},
+                          /*null_count=*/num_output_runs - num_valid_runs);
+
+      output_array_data =
+          ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                          {std::move(run_ends_data), std::move(values_data)},
+                          /*null_count=*/0);
+    }
+
+    if (input_length > 0) {
+      // Initialize the output pointers
+      auto* output_run_ends =
+          output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+      auto* output_validity =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(0);
+      auto* output_values =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(1);
+
+      if constexpr (has_validity_buffer) {
+        // Clear last byte in validity buffer to ensure padding bits are zeroed
+        output_validity[validity_buffer_size - 1] = 0;
+      }
+
+      // Second pass: write the runs
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> writing_loop(
+          input_length, input_offset, input_validity, input_values, output_validity,
+          output_values, output_run_ends);
+      [[maybe_unused]] int64_t num_written_runs = writing_loop.WriteEncodedRuns();
+      DCHECK_EQ(num_written_runs, num_output_runs);
+    }
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+template <typename RunEndType>
+Status RunEndEncodeNullArray(KernelContext* ctx, const ArraySpan& input_array,
+                             ExecResult* output) {
+  using RunEndCType = typename RunEndType::c_type;
+
+  const int64_t input_length = input_array.length;
+  auto input_array_type = input_array.type->GetSharedPtr();
+  DCHECK(input_array_type->id() == Type::NA);
+
+  int64_t num_output_runs = 0;
+  if (input_length > 0) {
+    // Abort if run-end type cannot hold the input length
+    RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_array.length));
+    num_output_runs = 1;
+  }
+
+  // Allocate the output array data
+  std::shared_ptr<ArrayData> output_array_data;
+  {
+    ARROW_ASSIGN_OR_RAISE(
+        auto run_ends_buffer,
+        AllocateBuffer(num_output_runs * RunEndType().bit_width(), ctx->memory_pool()));
+
+    auto ree_type = std::make_shared<RunEndEncodedType>(std::make_shared<RunEndType>(),
+                                                        input_array_type);
+    auto run_ends_data = ArrayData::Make(std::make_shared<RunEndType>(), num_output_runs,
+                                         {NULLPTR, std::move(run_ends_buffer)},
+                                         /*null_count=*/0);
+    auto values_data = ArrayData::Make(input_array_type, num_output_runs, {NULLPTR},
+                                       /*null_count=*/num_output_runs);
+
+    output_array_data =
+        ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                        {std::move(run_ends_data), std::move(values_data)},
+                        /*null_count=*/0);
+  }
+
+  if (input_length > 0) {
+    auto* output_run_ends =
+        output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+
+    // Write the single run-end this REE has
+    output_run_ends[0] = static_cast<RunEndCType>(input_length);
+  }
+
+  output->value = std::move(output_array_data);
+  return Status::OK();
+}
+
+template <typename ValueType>
+struct RunEndEncodeExec {
+  template <typename RunEndType>
+  static Status DoExec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    DCHECK(span.values[0].is_array());
+    const auto& input_array = span.values[0].array;
+    if constexpr (ValueType::type_id == Type::NA) {
+      return RunEndEncodeNullArray<RunEndType>(ctx, input_array, result);
+    } else {
+      const bool has_validity_buffer = input_array.MayHaveNulls();
+      if (has_validity_buffer) {
+        return RunEndEncodeImpl<RunEndType, ValueType, true>(ctx, input_array, result)
+            .Exec();
+      }
+      return RunEndEncodeImpl<RunEndType, ValueType, false>(ctx, input_array, result)
+          .Exec();
+    }
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    auto state = checked_cast<const RunEndEncondingState*>(ctx->state());
+    switch (state->run_end_type->id()) {
+      case Type::INT16:
+        return DoExec<Int16Type>(ctx, span, result);
+      case Type::INT32:
+        return DoExec<Int32Type>(ctx, span, result);
+      case Type::INT64:
+        return DoExec<Int64Type>(ctx, span, result);
+      default:
+        break;
+    }
+    return Status::Invalid("Invalid run end type: ", *state->run_end_type);
+  }
+};
+
+Result<std::unique_ptr<KernelState>> RunEndEncodeInit(KernelContext*,
+                                                      const KernelInitArgs& args) {
+  auto options = checked_cast<const RunEndEncodeOptions*>(args.options);
+  return std::make_unique<RunEndEncondingState>(options->run_end_type);
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndDecodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const ArraySpan& input_array_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+  int64_t values_offset_;
+
+  uint8_t* output_validity_;
+  void* output_values_;
+
+ public:
+  explicit RunEndDecodingLoop(const ArraySpan& input_array, ArrayData* output_array_data)
+      : input_array_(input_array) {
+    const ArraySpan& values = ree_util::ValuesArray(input_array);
+    input_validity_ = values.buffers[0].data;
+    input_values_ = values.buffers[1].data;
+    values_offset_ = values.offset;
+
+    output_validity_ = output_array_data->template GetMutableValues<uint8_t>(0);
+    output_values_ = output_array_data->template GetMutableValues<CType>(1);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteRun(int64_t write_offset, int64_t run_length, bool valid,
+                       CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteRun(
+        output_validity_, output_values_, write_offset, run_length, valid, value);
+  }
+
+ public:
+  /// \brief Expand all runs into the output array
+  ///
+  /// \return the number of non-null values written.
+  ARROW_NOINLINE int64_t ExpandAllRuns() {
+    // Ensure padding is zeroed in validity bitmap
+    if constexpr (has_validity_buffer) {
+      const int64_t validity_buffer_size = bit_util::BytesForBits(input_array_.length);
+      output_validity_[validity_buffer_size - 1] = 0;
+    }
+
+    const ree_util::RunEndEncodedArraySpan<RunEndCType> ree_array_span(input_array_);
+    int64_t write_offset = 0;
+    int64_t output_valid_count = 0;
+    for (auto it = ree_array_span.begin(); it != ree_array_span.end(); ++it) {
+      const int64_t read_offset = values_offset_ + it.index_into_array();
+      const int64_t run_length = it.run_length();
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+      WriteRun(write_offset, run_length, valid, value);
+      write_offset += run_length;
+      output_valid_count += valid ? run_length : 0;
+    }
+    DCHECK(write_offset == ree_array_span.length());
+    return output_valid_count;
+  }
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndDecodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndDecodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+ public:
+  Status Exec() {
+    const auto* ree_type = checked_cast<const RunEndEncodedType*>(input_array_.type);
+    const int64_t length = input_array_.length;
+    std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+    if constexpr (has_validity_buffer) {
+      // in bytes
+      int64_t validity_buffer_size = 0;
+      validity_buffer_size = bit_util::BytesForBits(length);
+      ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                            AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+    }
+    ARROW_ASSIGN_OR_RAISE(
+        auto values_buffer,
+        AllocateBuffer(
+            bit_util::BytesForBits(length * ree_type->value_type()->bit_width()),
+            ctx_->memory_pool()));
+
+    auto output_array_data =
+        ArrayData::Make(ree_type->value_type(), length,
+                        {std::move(validity_buffer), std::move(values_buffer)},
+                        /*child_data=*/std::vector<std::shared_ptr<ArrayData>>{});
+
+    int64_t output_null_count = 0;
+    if (length > 0) {
+      RunEndDecodingLoop<RunEndType, ValueType, has_validity_buffer> loop(
+          input_array_, output_array_data.get());
+      output_null_count = length - loop.ExpandAllRuns();
+    }
+    output_array_data->null_count = output_null_count;
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+Status RunEndDecodeNullREEArray(KernelContext* ctx, const ArraySpan& input_array,
+                                ExecResult* out) {
+  auto ree_type = checked_cast<const RunEndEncodedType*>(input_array.type);
+  ARROW_ASSIGN_OR_RAISE(auto output_array,
+                        arrow::MakeArrayOfNull(ree_type->value_type(), input_array.length,
+                                               ctx->memory_pool()));
+  out->value = output_array->data();
+  return Status::OK();
+}
+
+template <typename ValueType>
+struct RunEndDecodeExec {
+  template <typename RunEndType>
+  static Status DoExec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    DCHECK(span.values[0].is_array());
+    auto& input_array = span.values[0].array;
+    if constexpr (ValueType::type_id == Type::NA) {
+      return RunEndDecodeNullREEArray(ctx, input_array, result);
+    } else {
+      const bool has_validity_buffer = ree_util::ValuesArray(input_array).MayHaveNulls();
+      if (has_validity_buffer) {
+        return RunEndDecodeImpl<RunEndType, ValueType, true>(ctx, input_array, result)
+            .Exec();
+      }
+      return RunEndDecodeImpl<RunEndType, ValueType, false>(ctx, input_array, result)
+          .Exec();
+    }
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    const auto& ree_type = checked_cast<const RunEndEncodedType*>(span.values[0].type());
+    switch (ree_type->run_end_type()->id()) {
+      case Type::INT16:
+        return DoExec<Int16Type>(ctx, span, result);
+      case Type::INT32:
+        return DoExec<Int32Type>(ctx, span, result);
+      case Type::INT64:
+        return DoExec<Int64Type>(ctx, span, result);
+      default:
+        break;
+    }
+    return Status::Invalid("Invalid run end type: ", *ree_type->run_end_type());
+  }
+};
+
+static const FunctionDoc run_end_encode_doc(
+    "Run-end encode array", ("Return a run-end encoded version of the input array."),
+    {"array"}, "RunEndEncodeOptions", true);
+static const FunctionDoc run_end_decode_doc(
+    "Decode run-end encoded array",
+    ("Return a decoded version of a run-end encoded input array."), {"array"});
+
+static Result<TypeHolder> VectorRunEndEncodedResolver(
+    KernelContext* ctx, const std::vector<TypeHolder>& input_types) {
+  auto state = checked_cast<const RunEndEncondingState*>(ctx->state());
+  return TypeHolder(std::make_shared<RunEndEncodedType>(state->run_end_type,
+                                                        input_types[0].GetSharedPtr()));
+}
+
+void RegisterVectorRunEndEncode(FunctionRegistry* registry) {
+  auto function = std::make_shared<VectorFunction>("run_end_encode", Arity::Unary(),
+                                                   run_end_encode_doc);
+
+  auto add_kernel = [&function](const std::shared_ptr<DataType>& ty) {
+    auto sig =
+        KernelSignature::Make({InputType(ty)}, OutputType(VectorRunEndEncodedResolver));
+    auto exec = GenerateTypeAgnosticPrimitive<RunEndEncodeExec>(ty);
+    VectorKernel kernel(sig, exec, RunEndEncodeInit);
+    // A REE has null_count=0, so no need to allocate a validity bitmap for them.
+    kernel.null_handling = NullHandling::OUTPUT_NOT_NULL;
+    DCHECK_OK(function->AddKernel(std::move(kernel)));
+  };
+
+  for (const auto& ty : NumericTypes()) {
+    add_kernel(ty);
+  }
+  add_kernel(boolean());
+  add_kernel(null());
+  // TODO(GH-34195): Add support for more types
+
+  DCHECK_OK(registry->AddFunction(std::move(function)));
+}
+
+void RegisterVectorRunEndDecode(FunctionRegistry* registry) {
+  auto function = std::make_shared<VectorFunction>("run_end_decode", Arity::Unary(),
+                                                   run_end_decode_doc);
+
+  auto add_kernel = [&function](const std::shared_ptr<DataType>& ty) {
+    for (const auto& run_end_type : {int16(), int32(), int64()}) {
+      auto exec = GenerateTypeAgnosticPrimitive<RunEndDecodeExec>(ty);
+      auto input_type = std::make_shared<RunEndEncodedType>(run_end_type, ty);
+      auto sig = KernelSignature::Make({InputType(input_type)}, OutputType({ty}));
+      VectorKernel kernel(sig, exec);
+      DCHECK_OK(function->AddKernel(std::move(kernel)));
+    }
+  };

Review Comment:
   Sure, I will add the comment. The decision was conscious. I've done something similar in the builder (appending a slice can leave two runs with the same value). The complexity that handling these cases introduce can kill performance in some cases and give many surprises to people using the library IMHO. Not worth it for the rather small memory savings.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130215954


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),

Review Comment:
   Oh! yes :grimacing: 



-- 
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] zeroshade merged pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade merged PR #34195:
URL: https://github.com/apache/arrow/pull/34195


-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1129993498


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};

Review Comment:
   same comment as for the Read impl



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),
+                                           ctx_->memory_pool()));
+
+      auto ree_type = std::make_shared<RunEndEncodedType>(
+          std::make_shared<RunEndType>(), input_array_.type->GetSharedPtr());
+      auto run_ends_data =
+          ArrayData::Make(ree_type->run_end_type(), num_output_runs,
+                          {NULLPTR, std::move(run_ends_buffer)}, /*null_count=*/0);
+      auto values_data =
+          ArrayData::Make(ree_type->value_type(), num_output_runs,
+                          {std::move(validity_buffer), std::move(values_buffer)},
+                          /*null_count=*/num_output_runs - num_valid_runs);
+
+      output_array_data =
+          ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                          {std::move(run_ends_data), std::move(values_data)},
+                          /*null_count=*/0);
+    }
+
+    if (input_length > 0) {
+      // Initialize the output pointers
+      auto* output_run_ends =
+          output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+      auto* output_validity =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(0);
+      auto* output_values =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(1);
+
+      if constexpr (has_validity_buffer) {
+        // Clear last byte in validity buffer to ensure padding bits are zeroed
+        output_validity[validity_buffer_size - 1] = 0;
+      }
+
+      // Second pass: write the runs
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> writing_loop(
+          input_length, input_offset, input_validity, input_values, output_validity,
+          output_values, output_run_ends);
+      [[maybe_unused]] int64_t num_written_runs = writing_loop.WriteEncodedRuns();
+      DCHECK_EQ(num_written_runs, num_output_runs);
+    }
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+template <typename RunEndType>
+Status RunEndEncodeNullArray(KernelContext* ctx, const ArraySpan& input_array,
+                             ExecResult* output) {
+  using RunEndCType = typename RunEndType::c_type;
+
+  const int64_t input_length = input_array.length;
+  auto input_array_type = input_array.type->GetSharedPtr();
+  DCHECK(input_array_type->id() == Type::NA);
+
+  int64_t num_output_runs = 0;
+  if (input_length > 0) {

Review Comment:
   same as above



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   you can just use `byte_width()`  instead of having to `BytesForBits` the result here.



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;

Review Comment:
   Instead of using `enable_if` this can be cleaned and simplified:
   
   ```c++
   template <typename ArrowType, bool has_validity_buffer>
   struct ReadValueImpl {
       static_assert(has_c_type<T>::value);
       using CType = typename ArrowType::c_type;
   
       [[nodiscard]] bool ReadValue(.....) const {
           if constexpr (std::is_same_v(ArrowType, BooleanType)) {
                   // bool specialization
                   if constexpr (has_validity_buffer) {
                           // bool w/validity bitmap
                   } else {
                           // bool w/o validity bitmap
                   }
           } else {
                  // non-bool types
           }
       }
   };
   ```
   
   And it should result in pretty much the same compiled output without the need for the explicit `enable_if` complexity. Even if it doesn't make sense to simplify the overloads, you should still switch the enable_if to just a static_assert instead.



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }

Review Comment:
   because of the way Arrow is structured, technically there's no detriment to just doing this unconditionally and removing the `if (valid)` portion, eliminating a branch



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),

Review Comment:
   this should be `byte_width()` right? Same in all the other spots you allocate for



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {

Review Comment:
   rather than having `if (input_length > 0)` multiple times, how about just doing
   
   ```c++
   if (input_length <= 0) {
       // return empty result
   }
   
   // rest of code...
   ```
   
   to simplify things



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),
+                                           ctx_->memory_pool()));
+
+      auto ree_type = std::make_shared<RunEndEncodedType>(
+          std::make_shared<RunEndType>(), input_array_.type->GetSharedPtr());
+      auto run_ends_data =
+          ArrayData::Make(ree_type->run_end_type(), num_output_runs,
+                          {NULLPTR, std::move(run_ends_buffer)}, /*null_count=*/0);
+      auto values_data =
+          ArrayData::Make(ree_type->value_type(), num_output_runs,
+                          {std::move(validity_buffer), std::move(values_buffer)},
+                          /*null_count=*/num_output_runs - num_valid_runs);
+
+      output_array_data =
+          ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                          {std::move(run_ends_data), std::move(values_data)},
+                          /*null_count=*/0);
+    }
+
+    if (input_length > 0) {
+      // Initialize the output pointers
+      auto* output_run_ends =
+          output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+      auto* output_validity =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(0);
+      auto* output_values =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(1);
+
+      if constexpr (has_validity_buffer) {
+        // Clear last byte in validity buffer to ensure padding bits are zeroed
+        output_validity[validity_buffer_size - 1] = 0;
+      }
+
+      // Second pass: write the runs
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> writing_loop(
+          input_length, input_offset, input_validity, input_values, output_validity,
+          output_values, output_run_ends);
+      [[maybe_unused]] int64_t num_written_runs = writing_loop.WriteEncodedRuns();
+      DCHECK_EQ(num_written_runs, num_output_runs);
+    }
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+template <typename RunEndType>
+Status RunEndEncodeNullArray(KernelContext* ctx, const ArraySpan& input_array,
+                             ExecResult* output) {
+  using RunEndCType = typename RunEndType::c_type;
+
+  const int64_t input_length = input_array.length;
+  auto input_array_type = input_array.type->GetSharedPtr();
+  DCHECK(input_array_type->id() == Type::NA);
+
+  int64_t num_output_runs = 0;
+  if (input_length > 0) {
+    // Abort if run-end type cannot hold the input length
+    RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_array.length));
+    num_output_runs = 1;
+  }
+
+  // Allocate the output array data
+  std::shared_ptr<ArrayData> output_array_data;
+  {
+    ARROW_ASSIGN_OR_RAISE(
+        auto run_ends_buffer,
+        AllocateBuffer(num_output_runs * RunEndType().bit_width(), ctx->memory_pool()));
+
+    auto ree_type = std::make_shared<RunEndEncodedType>(std::make_shared<RunEndType>(),
+                                                        input_array_type);
+    auto run_ends_data = ArrayData::Make(std::make_shared<RunEndType>(), num_output_runs,
+                                         {NULLPTR, std::move(run_ends_buffer)},
+                                         /*null_count=*/0);
+    auto values_data = ArrayData::Make(input_array_type, num_output_runs, {NULLPTR},
+                                       /*null_count=*/num_output_runs);
+
+    output_array_data =
+        ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                        {std::move(run_ends_data), std::move(values_data)},
+                        /*null_count=*/0);
+  }
+
+  if (input_length > 0) {
+    auto* output_run_ends =
+        output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+
+    // Write the single run-end this REE has
+    output_run_ends[0] = static_cast<RunEndCType>(input_length);
+  }
+
+  output->value = std::move(output_array_data);
+  return Status::OK();
+}
+
+template <typename ValueType>
+struct RunEndEncodeExec {
+  template <typename RunEndType>
+  static Status DoExec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    DCHECK(span.values[0].is_array());
+    const auto& input_array = span.values[0].array;
+    if constexpr (ValueType::type_id == Type::NA) {
+      return RunEndEncodeNullArray<RunEndType>(ctx, input_array, result);
+    } else {
+      const bool has_validity_buffer = input_array.MayHaveNulls();
+      if (has_validity_buffer) {
+        return RunEndEncodeImpl<RunEndType, ValueType, true>(ctx, input_array, result)
+            .Exec();
+      }
+      return RunEndEncodeImpl<RunEndType, ValueType, false>(ctx, input_array, result)
+          .Exec();
+    }
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    auto state = checked_cast<const RunEndEncondingState*>(ctx->state());
+    switch (state->run_end_type->id()) {
+      case Type::INT16:
+        return DoExec<Int16Type>(ctx, span, result);
+      case Type::INT32:
+        return DoExec<Int32Type>(ctx, span, result);
+      case Type::INT64:
+        return DoExec<Int64Type>(ctx, span, result);
+      default:
+        break;
+    }
+    return Status::Invalid("Invalid run end type: ", *state->run_end_type);
+  }
+};
+
+Result<std::unique_ptr<KernelState>> RunEndEncodeInit(KernelContext*,
+                                                      const KernelInitArgs& args) {
+  auto options = checked_cast<const RunEndEncodeOptions*>(args.options);
+  return std::make_unique<RunEndEncondingState>(options->run_end_type);
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndDecodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const ArraySpan& input_array_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+  int64_t values_offset_;
+
+  uint8_t* output_validity_;
+  void* output_values_;
+
+ public:
+  explicit RunEndDecodingLoop(const ArraySpan& input_array, ArrayData* output_array_data)
+      : input_array_(input_array) {
+    const ArraySpan& values = ree_util::ValuesArray(input_array);
+    input_validity_ = values.buffers[0].data;
+    input_values_ = values.buffers[1].data;
+    values_offset_ = values.offset;
+
+    output_validity_ = output_array_data->template GetMutableValues<uint8_t>(0);
+    output_values_ = output_array_data->template GetMutableValues<CType>(1);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteRun(int64_t write_offset, int64_t run_length, bool valid,
+                       CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteRun(
+        output_validity_, output_values_, write_offset, run_length, valid, value);
+  }
+
+ public:
+  /// \brief Expand all runs into the output array
+  ///
+  /// \return the number of non-null values written.
+  ARROW_NOINLINE int64_t ExpandAllRuns() {
+    // Ensure padding is zeroed in validity bitmap
+    if constexpr (has_validity_buffer) {
+      const int64_t validity_buffer_size = bit_util::BytesForBits(input_array_.length);
+      output_validity_[validity_buffer_size - 1] = 0;
+    }
+
+    const ree_util::RunEndEncodedArraySpan<RunEndCType> ree_array_span(input_array_);
+    int64_t write_offset = 0;
+    int64_t output_valid_count = 0;
+    for (auto it = ree_array_span.begin(); it != ree_array_span.end(); ++it) {
+      const int64_t read_offset = values_offset_ + it.index_into_array();
+      const int64_t run_length = it.run_length();
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+      WriteRun(write_offset, run_length, valid, value);
+      write_offset += run_length;
+      output_valid_count += valid ? run_length : 0;
+    }
+    DCHECK(write_offset == ree_array_span.length());
+    return output_valid_count;
+  }
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndDecodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndDecodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+ public:
+  Status Exec() {
+    const auto* ree_type = checked_cast<const RunEndEncodedType*>(input_array_.type);
+    const int64_t length = input_array_.length;
+    std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+    if constexpr (has_validity_buffer) {
+      // in bytes
+      int64_t validity_buffer_size = 0;
+      validity_buffer_size = bit_util::BytesForBits(length);
+      ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                            AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+    }
+    ARROW_ASSIGN_OR_RAISE(
+        auto values_buffer,
+        AllocateBuffer(
+            bit_util::BytesForBits(length * ree_type->value_type()->bit_width()),
+            ctx_->memory_pool()));
+
+    auto output_array_data =
+        ArrayData::Make(ree_type->value_type(), length,
+                        {std::move(validity_buffer), std::move(values_buffer)},
+                        /*child_data=*/std::vector<std::shared_ptr<ArrayData>>{});
+
+    int64_t output_null_count = 0;
+    if (length > 0) {
+      RunEndDecodingLoop<RunEndType, ValueType, has_validity_buffer> loop(
+          input_array_, output_array_data.get());
+      output_null_count = length - loop.ExpandAllRuns();
+    }
+    output_array_data->null_count = output_null_count;
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+Status RunEndDecodeNullREEArray(KernelContext* ctx, const ArraySpan& input_array,
+                                ExecResult* out) {
+  auto ree_type = checked_cast<const RunEndEncodedType*>(input_array.type);
+  ARROW_ASSIGN_OR_RAISE(auto output_array,
+                        arrow::MakeArrayOfNull(ree_type->value_type(), input_array.length,
+                                               ctx->memory_pool()));
+  out->value = output_array->data();
+  return Status::OK();
+}
+
+template <typename ValueType>
+struct RunEndDecodeExec {
+  template <typename RunEndType>
+  static Status DoExec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    DCHECK(span.values[0].is_array());
+    auto& input_array = span.values[0].array;
+    if constexpr (ValueType::type_id == Type::NA) {
+      return RunEndDecodeNullREEArray(ctx, input_array, result);
+    } else {
+      const bool has_validity_buffer = ree_util::ValuesArray(input_array).MayHaveNulls();
+      if (has_validity_buffer) {
+        return RunEndDecodeImpl<RunEndType, ValueType, true>(ctx, input_array, result)
+            .Exec();
+      }
+      return RunEndDecodeImpl<RunEndType, ValueType, false>(ctx, input_array, result)
+          .Exec();
+    }
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    const auto& ree_type = checked_cast<const RunEndEncodedType*>(span.values[0].type());
+    switch (ree_type->run_end_type()->id()) {
+      case Type::INT16:
+        return DoExec<Int16Type>(ctx, span, result);
+      case Type::INT32:
+        return DoExec<Int32Type>(ctx, span, result);
+      case Type::INT64:
+        return DoExec<Int64Type>(ctx, span, result);
+      default:
+        break;
+    }
+    return Status::Invalid("Invalid run end type: ", *ree_type->run_end_type());
+  }
+};
+
+static const FunctionDoc run_end_encode_doc(
+    "Run-end encode array", ("Return a run-end encoded version of the input array."),
+    {"array"}, "RunEndEncodeOptions", true);
+static const FunctionDoc run_end_decode_doc(
+    "Decode run-end encoded array",
+    ("Return a decoded version of a run-end encoded input array."), {"array"});
+
+static Result<TypeHolder> VectorRunEndEncodedResolver(
+    KernelContext* ctx, const std::vector<TypeHolder>& input_types) {
+  auto state = checked_cast<const RunEndEncondingState*>(ctx->state());
+  return TypeHolder(std::make_shared<RunEndEncodedType>(state->run_end_type,
+                                                        input_types[0].GetSharedPtr()));
+}
+
+void RegisterVectorRunEndEncode(FunctionRegistry* registry) {
+  auto function = std::make_shared<VectorFunction>("run_end_encode", Arity::Unary(),
+                                                   run_end_encode_doc);
+
+  auto add_kernel = [&function](const std::shared_ptr<DataType>& ty) {
+    auto sig =
+        KernelSignature::Make({InputType(ty)}, OutputType(VectorRunEndEncodedResolver));
+    auto exec = GenerateTypeAgnosticPrimitive<RunEndEncodeExec>(ty);
+    VectorKernel kernel(sig, exec, RunEndEncodeInit);
+    // A REE has null_count=0, so no need to allocate a validity bitmap for them.
+    kernel.null_handling = NullHandling::OUTPUT_NOT_NULL;
+    DCHECK_OK(function->AddKernel(std::move(kernel)));
+  };
+
+  for (const auto& ty : NumericTypes()) {
+    add_kernel(ty);
+  }
+  add_kernel(boolean());
+  add_kernel(null());
+  // TODO(GH-34195): Add support for more types
+
+  DCHECK_OK(registry->AddFunction(std::move(function)));
+}
+
+void RegisterVectorRunEndDecode(FunctionRegistry* registry) {
+  auto function = std::make_shared<VectorFunction>("run_end_decode", Arity::Unary(),
+                                                   run_end_decode_doc);
+
+  auto add_kernel = [&function](const std::shared_ptr<DataType>& ty) {
+    for (const auto& run_end_type : {int16(), int32(), int64()}) {
+      auto exec = GenerateTypeAgnosticPrimitive<RunEndDecodeExec>(ty);
+      auto input_type = std::make_shared<RunEndEncodedType>(run_end_type, ty);
+      auto sig = KernelSignature::Make({InputType(input_type)}, OutputType({ty}));
+      VectorKernel kernel(sig, exec);
+      DCHECK_OK(function->AddKernel(std::move(kernel)));
+    }
+  };

Review Comment:
   An interesting question here is whether or not we want to keep the function to operating chunk-wise or let it benefit from potentially condensing runs that go across chunks? Currently it looks like your implementation only operates chunk-wise, so if one chunk ends with a value and the next chunk starts with the same value, they won't get condensed into a single run in the output but rather each will be a run in their own chunk.
   
   I'm not saying that's wrong, just wondering if it might make sense to instead utilize the finalize function setup and allow that condensing across chunks. Obviously there's pros and cons to doing so as far as execution time vs memory. Honestly, I think there's only very few cases where it would be truly beneficial and I think that operating chunk-wise like you did is the better choice. Should we at least put a comment or something to the effect so that we remember why chose to do it this way?



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }

Review Comment:
   technically this would never catch if they manage to overflow an int64 run-end type and it would just roll on forward. You should probably add `|| input_length < 0` to the condition so that you catch an int64 overflow too. At a minimum, I'd add a `DCHECK`



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130208696


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;

Review Comment:
   Removing the `BooleanType` specializations will be nice indeed, but the enable_if was done in preparation to support non-primitive types. I'm following the same pattern of `visit_data_inline.h` [1].
   
   [1] https://github.com/apache/arrow/blob/main/cpp/src/arrow/visit_data_inline.h#L37



-- 
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] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1131284388


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;

Review Comment:
   We can leave it as is if other reviewers are happy. Possibly file an issue to continue consolidating kernel utilities like this together where possible



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

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

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


[GitHub] [arrow] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1128333025


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {

Review Comment:
   I decided against this solution and added a commit that reverts 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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130230911


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }

Review Comment:
   Done.



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

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

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1120278602


##########
cpp/src/arrow/compute/api_vector.h:
##########
@@ -577,6 +586,30 @@ Result<Datum> DictionaryEncode(
     const DictionaryEncodeOptions& options = DictionaryEncodeOptions::Defaults(),
     ExecContext* ctx = NULLPTR);
 
+/// \brief Run-end-encode values in an array-like object
+///
+/// \param[in] value array-like input
+/// \param[in] options configures encoding behavior
+/// \param[in] ctx the function execution context, optional
+/// \return result with same shape and type as input

Review Comment:
   same shape, different type, right?



##########
cpp/src/arrow/compute/api_vector.h:
##########
@@ -577,6 +586,30 @@ Result<Datum> DictionaryEncode(
     const DictionaryEncodeOptions& options = DictionaryEncodeOptions::Defaults(),
     ExecContext* ctx = NULLPTR);
 
+/// \brief Run-end-encode values in an array-like object
+///
+/// \param[in] value array-like input
+/// \param[in] options configures encoding behavior
+/// \param[in] ctx the function execution context, optional
+/// \return result with same shape and type as input
+///
+/// \since 12.0.0
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> RunEndEncode(const Datum& value, const RunEndEncodeOptions& options,
+                           ExecContext* ctx = NULLPTR);
+
+/// \brief Decode a Run-End Encoded array to a plain array
+///
+/// \param[in] value run-end-encoded input
+/// \param[in] ctx the function execution context, optional
+/// \return result with same shape and type as input

Review Comment:
   ditto



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;

Review Comment:
   nit: `string` isn't very descriptive, maybe `description` if an instance of this is meant to be a test case?



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));
+    ARROW_CHECK_OK(builder->AppendArraySlice(ArraySpan(*child), 0, child->length));
+    array->child_data[1] = builder->Finish().ValueOrDie()->Slice(offset)->data();
+  }
+};
+
+TEST_P(TestRunEndEncodeDecode, EncodeDecodeArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  auto run_ends_array = MakeArray(encoded->child_data[0]);
+  auto values_array = MakeArray(encoded->child_data[1]);
+  ASSERT_OK(MakeArray(encoded)->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*ArrayFromJSON(run_ends_type, data.expected_run_ends_json),
+                      *run_ends_array);
+  ASSERT_ARRAYS_EQUAL(*values_array, *data.expected_values);
+  ASSERT_EQ(encoded->buffers.size(), 1);
+  ASSERT_EQ(encoded->buffers[0], NULLPTR);
+  ASSERT_EQ(encoded->child_data.size(), 2);
+  ASSERT_EQ(run_ends_array->data()->buffers[0], NULLPTR);
+  ASSERT_EQ(run_ends_array->length(), data.expected_values->length());
+  ASSERT_EQ(run_ends_array->offset(), 0);
+  ASSERT_EQ(encoded->length, data.input->length());
+  ASSERT_EQ(encoded->offset, 0);
+  ASSERT_EQ(*encoded->type, RunEndEncodedType(run_ends_type, data.input->type()));
+  ASSERT_EQ(encoded->null_count, 0);
+
+  ASSERT_OK_AND_ASSIGN(Datum decoded_datum, RunEndDecode(encoded));
+  auto decoded = decoded_datum.make_array();
+  ASSERT_OK(decoded->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*decoded, *data.input);
+}
+
+// Encoding an input with an offset results in a completely new encoded array without an
+// offset. This means The EncodeDecodeArray test will never actually decode an array
+// with an offset, even though we have inputs with offsets. This test slices one element
+// off the encoded array and decodes that.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffset) {
+  auto [data, run_ends_type] = GetParam();
+  if (data.input->length() == 0) {
+    return;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first,
+                       RunEndDecode(encoded->Slice(1, encoded->length - 1)));
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_last,
+                       RunEndDecode(encoded->Slice(0, encoded->length - 1)));
+  auto array_without_first = datum_without_first.make_array();
+  auto array_without_last = datum_without_last.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_OK(array_without_last->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input->Slice(1));
+  ASSERT_ARRAYS_EQUAL(*array_without_last,
+                      *data.input->Slice(0, data.input->length() - 1));
+}
+
+// This test creates an run-end encoded array with an offset in the child array, which
+// removes the first run in the test data.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffsetInChildArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  this->AddArtificialOffsetInChildArray(encoded.get(), 100);
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first, RunEndDecode(encoded));
+  auto array_without_first = datum_without_first.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input);
+}
+
+INSTANTIATE_TEST_SUITE_P(

Review Comment:
   Is it possible to test scalars and chunked arrays here, too?



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl {

Review Comment:
   Is there any possibility to consolidate with https://github.com/apache/arrow/blob/f74d5285803c23e6f553d655ea5f184d06d26607/cpp/src/arrow/compute/kernels/copy_data_internal.h



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));

Review Comment:
   You can use ASSERT_OK and ASSERT_OK_AND_ASSIGN
   
   Then you can call this as `ASSERT_NO_FATAL_FAILURE(AddArtificialOffsetInChildArray(...))`



##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {

Review Comment:
   Is it possible to use one of the existing typed visitors instead of a handrolled loop?



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1132838772


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   oy, well that's inconvenient....



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1132841273


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {

Review Comment:
   would it make sense to put the output allocating into a function which you can just call with the 0 and do the early return for input_length == 0? 
   
   I agree that the repeated `if (input_length > 0)` is preferable to duplicating the whole block. I'd be fine with either leaving it as is or shifting the allocation of the output to a function which is called during the early return. Up to you.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130256770


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),
+                                           ctx_->memory_pool()));
+
+      auto ree_type = std::make_shared<RunEndEncodedType>(
+          std::make_shared<RunEndType>(), input_array_.type->GetSharedPtr());
+      auto run_ends_data =
+          ArrayData::Make(ree_type->run_end_type(), num_output_runs,
+                          {NULLPTR, std::move(run_ends_buffer)}, /*null_count=*/0);
+      auto values_data =
+          ArrayData::Make(ree_type->value_type(), num_output_runs,
+                          {std::move(validity_buffer), std::move(values_buffer)},
+                          /*null_count=*/num_output_runs - num_valid_runs);
+
+      output_array_data =
+          ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                          {std::move(run_ends_data), std::move(values_data)},
+                          /*null_count=*/0);
+    }
+
+    if (input_length > 0) {
+      // Initialize the output pointers
+      auto* output_run_ends =
+          output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+      auto* output_validity =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(0);
+      auto* output_values =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(1);
+
+      if constexpr (has_validity_buffer) {
+        // Clear last byte in validity buffer to ensure padding bits are zeroed
+        output_validity[validity_buffer_size - 1] = 0;
+      }
+
+      // Second pass: write the runs
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> writing_loop(
+          input_length, input_offset, input_validity, input_values, output_validity,
+          output_values, output_run_ends);
+      [[maybe_unused]] int64_t num_written_runs = writing_loop.WriteEncodedRuns();
+      DCHECK_EQ(num_written_runs, num_output_runs);
+    }
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+template <typename RunEndType>
+Status RunEndEncodeNullArray(KernelContext* ctx, const ArraySpan& input_array,
+                             ExecResult* output) {
+  using RunEndCType = typename RunEndType::c_type;
+
+  const int64_t input_length = input_array.length;
+  auto input_array_type = input_array.type->GetSharedPtr();
+  DCHECK(input_array_type->id() == Type::NA);
+
+  int64_t num_output_runs = 0;
+  if (input_length > 0) {
+    // Abort if run-end type cannot hold the input length
+    RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_array.length));
+    num_output_runs = 1;
+  }
+
+  // Allocate the output array data
+  std::shared_ptr<ArrayData> output_array_data;
+  {
+    ARROW_ASSIGN_OR_RAISE(
+        auto run_ends_buffer,
+        AllocateBuffer(num_output_runs * RunEndType().bit_width(), ctx->memory_pool()));
+
+    auto ree_type = std::make_shared<RunEndEncodedType>(std::make_shared<RunEndType>(),
+                                                        input_array_type);
+    auto run_ends_data = ArrayData::Make(std::make_shared<RunEndType>(), num_output_runs,
+                                         {NULLPTR, std::move(run_ends_buffer)},
+                                         /*null_count=*/0);
+    auto values_data = ArrayData::Make(input_array_type, num_output_runs, {NULLPTR},
+                                       /*null_count=*/num_output_runs);
+
+    output_array_data =
+        ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                        {std::move(run_ends_data), std::move(values_data)},
+                        /*null_count=*/0);
+  }
+
+  if (input_length > 0) {
+    auto* output_run_ends =
+        output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+
+    // Write the single run-end this REE has
+    output_run_ends[0] = static_cast<RunEndCType>(input_length);
+  }
+
+  output->value = std::move(output_array_data);
+  return Status::OK();
+}
+
+template <typename ValueType>
+struct RunEndEncodeExec {
+  template <typename RunEndType>
+  static Status DoExec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    DCHECK(span.values[0].is_array());
+    const auto& input_array = span.values[0].array;
+    if constexpr (ValueType::type_id == Type::NA) {
+      return RunEndEncodeNullArray<RunEndType>(ctx, input_array, result);
+    } else {
+      const bool has_validity_buffer = input_array.MayHaveNulls();
+      if (has_validity_buffer) {
+        return RunEndEncodeImpl<RunEndType, ValueType, true>(ctx, input_array, result)
+            .Exec();
+      }
+      return RunEndEncodeImpl<RunEndType, ValueType, false>(ctx, input_array, result)
+          .Exec();
+    }
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    auto state = checked_cast<const RunEndEncondingState*>(ctx->state());
+    switch (state->run_end_type->id()) {
+      case Type::INT16:
+        return DoExec<Int16Type>(ctx, span, result);
+      case Type::INT32:
+        return DoExec<Int32Type>(ctx, span, result);
+      case Type::INT64:
+        return DoExec<Int64Type>(ctx, span, result);
+      default:
+        break;
+    }
+    return Status::Invalid("Invalid run end type: ", *state->run_end_type);
+  }
+};
+
+Result<std::unique_ptr<KernelState>> RunEndEncodeInit(KernelContext*,
+                                                      const KernelInitArgs& args) {
+  auto options = checked_cast<const RunEndEncodeOptions*>(args.options);
+  return std::make_unique<RunEndEncondingState>(options->run_end_type);
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndDecodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const ArraySpan& input_array_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+  int64_t values_offset_;
+
+  uint8_t* output_validity_;
+  void* output_values_;
+
+ public:
+  explicit RunEndDecodingLoop(const ArraySpan& input_array, ArrayData* output_array_data)
+      : input_array_(input_array) {
+    const ArraySpan& values = ree_util::ValuesArray(input_array);
+    input_validity_ = values.buffers[0].data;
+    input_values_ = values.buffers[1].data;
+    values_offset_ = values.offset;
+
+    output_validity_ = output_array_data->template GetMutableValues<uint8_t>(0);
+    output_values_ = output_array_data->template GetMutableValues<CType>(1);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteRun(int64_t write_offset, int64_t run_length, bool valid,
+                       CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteRun(
+        output_validity_, output_values_, write_offset, run_length, valid, value);
+  }
+
+ public:
+  /// \brief Expand all runs into the output array
+  ///
+  /// \return the number of non-null values written.
+  ARROW_NOINLINE int64_t ExpandAllRuns() {
+    // Ensure padding is zeroed in validity bitmap
+    if constexpr (has_validity_buffer) {
+      const int64_t validity_buffer_size = bit_util::BytesForBits(input_array_.length);
+      output_validity_[validity_buffer_size - 1] = 0;
+    }
+
+    const ree_util::RunEndEncodedArraySpan<RunEndCType> ree_array_span(input_array_);
+    int64_t write_offset = 0;
+    int64_t output_valid_count = 0;
+    for (auto it = ree_array_span.begin(); it != ree_array_span.end(); ++it) {
+      const int64_t read_offset = values_offset_ + it.index_into_array();
+      const int64_t run_length = it.run_length();
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+      WriteRun(write_offset, run_length, valid, value);
+      write_offset += run_length;
+      output_valid_count += valid ? run_length : 0;
+    }
+    DCHECK(write_offset == ree_array_span.length());
+    return output_valid_count;
+  }
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndDecodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndDecodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+ public:
+  Status Exec() {
+    const auto* ree_type = checked_cast<const RunEndEncodedType*>(input_array_.type);
+    const int64_t length = input_array_.length;
+    std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+    if constexpr (has_validity_buffer) {
+      // in bytes
+      int64_t validity_buffer_size = 0;
+      validity_buffer_size = bit_util::BytesForBits(length);
+      ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                            AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+    }
+    ARROW_ASSIGN_OR_RAISE(
+        auto values_buffer,
+        AllocateBuffer(
+            bit_util::BytesForBits(length * ree_type->value_type()->bit_width()),
+            ctx_->memory_pool()));
+
+    auto output_array_data =
+        ArrayData::Make(ree_type->value_type(), length,
+                        {std::move(validity_buffer), std::move(values_buffer)},
+                        /*child_data=*/std::vector<std::shared_ptr<ArrayData>>{});
+
+    int64_t output_null_count = 0;
+    if (length > 0) {
+      RunEndDecodingLoop<RunEndType, ValueType, has_validity_buffer> loop(
+          input_array_, output_array_data.get());
+      output_null_count = length - loop.ExpandAllRuns();
+    }
+    output_array_data->null_count = output_null_count;
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+Status RunEndDecodeNullREEArray(KernelContext* ctx, const ArraySpan& input_array,
+                                ExecResult* out) {
+  auto ree_type = checked_cast<const RunEndEncodedType*>(input_array.type);
+  ARROW_ASSIGN_OR_RAISE(auto output_array,
+                        arrow::MakeArrayOfNull(ree_type->value_type(), input_array.length,
+                                               ctx->memory_pool()));
+  out->value = output_array->data();
+  return Status::OK();
+}
+
+template <typename ValueType>
+struct RunEndDecodeExec {
+  template <typename RunEndType>
+  static Status DoExec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    DCHECK(span.values[0].is_array());
+    auto& input_array = span.values[0].array;
+    if constexpr (ValueType::type_id == Type::NA) {
+      return RunEndDecodeNullREEArray(ctx, input_array, result);
+    } else {
+      const bool has_validity_buffer = ree_util::ValuesArray(input_array).MayHaveNulls();
+      if (has_validity_buffer) {
+        return RunEndDecodeImpl<RunEndType, ValueType, true>(ctx, input_array, result)
+            .Exec();
+      }
+      return RunEndDecodeImpl<RunEndType, ValueType, false>(ctx, input_array, result)
+          .Exec();
+    }
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecSpan& span, ExecResult* result) {
+    const auto& ree_type = checked_cast<const RunEndEncodedType*>(span.values[0].type());
+    switch (ree_type->run_end_type()->id()) {
+      case Type::INT16:
+        return DoExec<Int16Type>(ctx, span, result);
+      case Type::INT32:
+        return DoExec<Int32Type>(ctx, span, result);
+      case Type::INT64:
+        return DoExec<Int64Type>(ctx, span, result);
+      default:
+        break;
+    }
+    return Status::Invalid("Invalid run end type: ", *ree_type->run_end_type());
+  }
+};
+
+static const FunctionDoc run_end_encode_doc(
+    "Run-end encode array", ("Return a run-end encoded version of the input array."),
+    {"array"}, "RunEndEncodeOptions", true);
+static const FunctionDoc run_end_decode_doc(
+    "Decode run-end encoded array",
+    ("Return a decoded version of a run-end encoded input array."), {"array"});
+
+static Result<TypeHolder> VectorRunEndEncodedResolver(
+    KernelContext* ctx, const std::vector<TypeHolder>& input_types) {
+  auto state = checked_cast<const RunEndEncondingState*>(ctx->state());
+  return TypeHolder(std::make_shared<RunEndEncodedType>(state->run_end_type,
+                                                        input_types[0].GetSharedPtr()));
+}
+
+void RegisterVectorRunEndEncode(FunctionRegistry* registry) {
+  auto function = std::make_shared<VectorFunction>("run_end_encode", Arity::Unary(),
+                                                   run_end_encode_doc);
+
+  auto add_kernel = [&function](const std::shared_ptr<DataType>& ty) {
+    auto sig =
+        KernelSignature::Make({InputType(ty)}, OutputType(VectorRunEndEncodedResolver));
+    auto exec = GenerateTypeAgnosticPrimitive<RunEndEncodeExec>(ty);
+    VectorKernel kernel(sig, exec, RunEndEncodeInit);
+    // A REE has null_count=0, so no need to allocate a validity bitmap for them.
+    kernel.null_handling = NullHandling::OUTPUT_NOT_NULL;
+    DCHECK_OK(function->AddKernel(std::move(kernel)));
+  };
+
+  for (const auto& ty : NumericTypes()) {
+    add_kernel(ty);
+  }
+  add_kernel(boolean());
+  add_kernel(null());
+  // TODO(GH-34195): Add support for more types
+
+  DCHECK_OK(registry->AddFunction(std::move(function)));
+}
+
+void RegisterVectorRunEndDecode(FunctionRegistry* registry) {
+  auto function = std::make_shared<VectorFunction>("run_end_decode", Arity::Unary(),
+                                                   run_end_decode_doc);
+
+  auto add_kernel = [&function](const std::shared_ptr<DataType>& ty) {
+    for (const auto& run_end_type : {int16(), int32(), int64()}) {
+      auto exec = GenerateTypeAgnosticPrimitive<RunEndDecodeExec>(ty);
+      auto input_type = std::make_shared<RunEndEncodedType>(run_end_type, ty);
+      auto sig = KernelSignature::Make({InputType(input_type)}, OutputType({ty}));
+      VectorKernel kernel(sig, exec);
+      DCHECK_OK(function->AddKernel(std::move(kernel)));
+    }
+  };

Review Comment:
   (the builder takes a slice of another ree array, so just concatenating blindly is much simpler)



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1126921108


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl {

Review Comment:
   Do you mean adding function to `copy_data_internal.h` classes that can copy + repeat values into array buffers?



-- 
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] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1127031892


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;

Review Comment:
   It seems things didn't change (looking at GitHub)?



-- 
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] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1127031164


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl {

Review Comment:
   Possibly - if nothing there seems useful then don't bother



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130222306


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }

Review Comment:
   Done.



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

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

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


[GitHub] [arrow] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130255227


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -623,6 +579,13 @@ void RegisterVectorRunEndEncode(FunctionRegistry* registry) {
   auto function = std::make_shared<VectorFunction>("run_end_encode", Arity::Unary(),
                                                    run_end_encode_doc);
 
+  // NOTE: When the input to run_end_encode() is a ChunkedArray, the output is also a
+  // ChunkedArray with the same number of chunks as the input. Each chunk in the output
+  // has the same logical length as the corresponding chunk in the input. This simplicity
+  // has a small downside: if a run of identical values crosses a chunk boundary, this run
+  // cannot be encoded as a single run in the output. This is a conscious trade-off as
+  // trying to solve this corner-case would complicate the implementation,
+  // require reallocations, and could create surprising behavior for users of this API.

Review Comment:
   @zeroshade what you think? Should this be part of the function documentation?



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1131232393


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   for a `BooleanType` you should use `AllocateBitmap` instead, right?
   
   `BytesForBits(num_output_runs * bit_width())` should be identical to `byte_width()` even in the boolean case, shouldn't 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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1126942080


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,654 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {

Review Comment:
   I think I can use `visit_data_inline.h` for some steps but not everything and the readability of the loops might become worse. I will try to do 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] lidavidm commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1127030916


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));
+    ARROW_CHECK_OK(builder->AppendArraySlice(ArraySpan(*child), 0, child->length));
+    array->child_data[1] = builder->Finish().ValueOrDie()->Slice(offset)->data();
+  }
+};
+
+TEST_P(TestRunEndEncodeDecode, EncodeDecodeArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  auto run_ends_array = MakeArray(encoded->child_data[0]);
+  auto values_array = MakeArray(encoded->child_data[1]);
+  ASSERT_OK(MakeArray(encoded)->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*ArrayFromJSON(run_ends_type, data.expected_run_ends_json),
+                      *run_ends_array);
+  ASSERT_ARRAYS_EQUAL(*values_array, *data.expected_values);
+  ASSERT_EQ(encoded->buffers.size(), 1);
+  ASSERT_EQ(encoded->buffers[0], NULLPTR);
+  ASSERT_EQ(encoded->child_data.size(), 2);
+  ASSERT_EQ(run_ends_array->data()->buffers[0], NULLPTR);
+  ASSERT_EQ(run_ends_array->length(), data.expected_values->length());
+  ASSERT_EQ(run_ends_array->offset(), 0);
+  ASSERT_EQ(encoded->length, data.input->length());
+  ASSERT_EQ(encoded->offset, 0);
+  ASSERT_EQ(*encoded->type, RunEndEncodedType(run_ends_type, data.input->type()));
+  ASSERT_EQ(encoded->null_count, 0);
+
+  ASSERT_OK_AND_ASSIGN(Datum decoded_datum, RunEndDecode(encoded));
+  auto decoded = decoded_datum.make_array();
+  ASSERT_OK(decoded->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*decoded, *data.input);
+}
+
+// Encoding an input with an offset results in a completely new encoded array without an
+// offset. This means The EncodeDecodeArray test will never actually decode an array
+// with an offset, even though we have inputs with offsets. This test slices one element
+// off the encoded array and decodes that.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffset) {
+  auto [data, run_ends_type] = GetParam();
+  if (data.input->length() == 0) {
+    return;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first,
+                       RunEndDecode(encoded->Slice(1, encoded->length - 1)));
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_last,
+                       RunEndDecode(encoded->Slice(0, encoded->length - 1)));
+  auto array_without_first = datum_without_first.make_array();
+  auto array_without_last = datum_without_last.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_OK(array_without_last->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input->Slice(1));
+  ASSERT_ARRAYS_EQUAL(*array_without_last,
+                      *data.input->Slice(0, data.input->length() - 1));
+}
+
+// This test creates an run-end encoded array with an offset in the child array, which
+// removes the first run in the test data.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffsetInChildArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  this->AddArtificialOffsetInChildArray(encoded.get(), 100);
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first, RunEndDecode(encoded));
+  auto array_without_first = datum_without_first.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input);
+}
+
+INSTANTIATE_TEST_SUITE_P(

Review Comment:
   Ah, whoops!



-- 
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 #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1458141966

   :warning: GitHub issue #32105 **has been automatically assigned in GitHub** to PR creator.


-- 
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] felipecrv commented on pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1447068955

   I just noticed the input type matcher isn't really necessary for these kernels, so if you want I can remove this commit from the PR and push again.


-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1134112053


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {

Review Comment:
   I extracted the allocation function and rewrote this to early-return on 0 length.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130216822


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),
+                                           ctx_->memory_pool()));
+
+      auto ree_type = std::make_shared<RunEndEncodedType>(
+          std::make_shared<RunEndType>(), input_array_.type->GetSharedPtr());
+      auto run_ends_data =
+          ArrayData::Make(ree_type->run_end_type(), num_output_runs,
+                          {NULLPTR, std::move(run_ends_buffer)}, /*null_count=*/0);
+      auto values_data =
+          ArrayData::Make(ree_type->value_type(), num_output_runs,
+                          {std::move(validity_buffer), std::move(values_buffer)},
+                          /*null_count=*/num_output_runs - num_valid_runs);
+
+      output_array_data =
+          ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                          {std::move(run_ends_data), std::move(values_data)},
+                          /*null_count=*/0);
+    }
+
+    if (input_length > 0) {
+      // Initialize the output pointers
+      auto* output_run_ends =
+          output_array_data->child_data[0]->template GetMutableValues<RunEndCType>(1);
+      auto* output_validity =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(0);
+      auto* output_values =
+          output_array_data->child_data[1]->template GetMutableValues<uint8_t>(1);
+
+      if constexpr (has_validity_buffer) {
+        // Clear last byte in validity buffer to ensure padding bits are zeroed
+        output_validity[validity_buffer_size - 1] = 0;
+      }
+
+      // Second pass: write the runs
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> writing_loop(
+          input_length, input_offset, input_validity, input_values, output_validity,
+          output_values, output_run_ends);
+      [[maybe_unused]] int64_t num_written_runs = writing_loop.WriteEncodedRuns();
+      DCHECK_EQ(num_written_runs, num_output_runs);
+    }
+
+    output_->value = std::move(output_array_data);
+    return Status::OK();
+  }
+};
+
+template <typename RunEndType>
+Status RunEndEncodeNullArray(KernelContext* ctx, const ArraySpan& input_array,
+                             ExecResult* output) {
+  using RunEndCType = typename RunEndType::c_type;
+
+  const int64_t input_length = input_array.length;
+  auto input_array_type = input_array.type->GetSharedPtr();
+  DCHECK(input_array_type->id() == Type::NA);
+
+  int64_t num_output_runs = 0;
+  if (input_length > 0) {
+    // Abort if run-end type cannot hold the input length
+    RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_array.length));
+    num_output_runs = 1;
+  }
+
+  // Allocate the output array data
+  std::shared_ptr<ArrayData> output_array_data;
+  {
+    ARROW_ASSIGN_OR_RAISE(
+        auto run_ends_buffer,
+        AllocateBuffer(num_output_runs * RunEndType().bit_width(), ctx->memory_pool()));
+
+    auto ree_type = std::make_shared<RunEndEncodedType>(std::make_shared<RunEndType>(),
+                                                        input_array_type);
+    auto run_ends_data = ArrayData::Make(std::make_shared<RunEndType>(), num_output_runs,
+                                         {NULLPTR, std::move(run_ends_buffer)},
+                                         /*null_count=*/0);
+    auto values_data = ArrayData::Make(input_array_type, num_output_runs, {NULLPTR},
+                                       /*null_count=*/num_output_runs);
+
+    output_array_data =
+        ArrayData::Make(std::move(ree_type), input_length, {NULLPTR},
+                        {std::move(run_ends_data), std::move(values_data)},
+                        /*null_count=*/0);
+  }

Review Comment:
   @zeroshade early-returning for the 0 case means repeating this block. Don't you think it's more elegant that this code can handle any length and as such doesn't have to be duplicated?



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130214275


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }

Review Comment:
   I will add the extra boolean check.



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1131237027


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -623,6 +579,13 @@ void RegisterVectorRunEndEncode(FunctionRegistry* registry) {
   auto function = std::make_shared<VectorFunction>("run_end_encode", Arity::Unary(),
                                                    run_end_encode_doc);
 
+  // NOTE: When the input to run_end_encode() is a ChunkedArray, the output is also a
+  // ChunkedArray with the same number of chunks as the input. Each chunk in the output
+  // has the same logical length as the corresponding chunk in the input. This simplicity
+  // has a small downside: if a run of identical values crosses a chunk boundary, this run
+  // cannot be encoded as a single run in the output. This is a conscious trade-off as
+  // trying to solve this corner-case would complicate the implementation,
+  // require reallocations, and could create surprising behavior for users of this API.

Review Comment:
   I think the comment in the code here is sufficient to me. It might be worthwhile to add a quick sentence in the function docs to say that if you pass a chunked array you can end up with runs with the same value at the end/start of consecutive chunks but this comment is enough to make me happy here.



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

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

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


[GitHub] [arrow] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1130236881


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,672 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct ReadValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+// Boolean w/o validity_bitmap
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+
+template <typename ArrowType, bool has_validity_buffer, typename Enable = void>
+struct WriteValueImpl {};
+
+// Numeric and primitive C-compatible types
+template <typename ArrowType, bool has_validity_buffer>
+struct WriteValueImpl<ArrowType, has_validity_buffer, enable_if_has_c_type<ArrowType>> {
+  using CType = typename ArrowType::c_type;
+
+  void WriteValue(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                  bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, 1, valid);
+    }
+    (reinterpret_cast<CType*>(output_values))[write_offset] = value;
+  }
+
+  void WriteRun(uint8_t* output_validity, void* output_values, int64_t write_offset,
+                int64_t run_length, bool valid, CType value) const {
+    if constexpr (has_validity_buffer) {
+      bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+    }
+    auto* output_values_c = reinterpret_cast<CType*>(output_values);
+    std::fill(output_values_c + write_offset, output_values_c + write_offset + run_length,
+              value);
+  }
+};
+
+// Boolean w/ validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, true>::WriteValue(uint8_t* output_validity,
+                                                   void* output_values,
+                                                   int64_t write_offset, bool valid,
+                                                   CType value) const {
+  bit_util::SetBitTo(output_validity, write_offset, valid);
+  if (valid) {
+    bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+  }
+}
+
+template <>
+void WriteValueImpl<BooleanType, true>::WriteRun(uint8_t* output_validity,
+                                                 void* output_values,
+                                                 int64_t write_offset, int64_t run_length,
+                                                 bool valid, CType value) const {
+  bit_util::SetBitsTo(output_validity, write_offset, run_length, valid);
+  if (valid) {
+    bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset,
+                        run_length, value);
+  }
+}
+
+// Boolean w/o validity_bitmap
+template <>
+void WriteValueImpl<BooleanType, false>::WriteValue(uint8_t*, void* output_values,
+                                                    int64_t write_offset, bool,
+                                                    CType value) const {
+  bit_util::SetBitTo(reinterpret_cast<uint8_t*>(output_values), write_offset, value);
+}
+
+template <>
+void WriteValueImpl<BooleanType, false>::WriteRun(uint8_t*, void* output_values,
+                                                  int64_t write_offset,
+                                                  int64_t run_length, bool,
+                                                  CType value) const {
+  bit_util::SetBitsTo(reinterpret_cast<uint8_t*>(output_values), write_offset, run_length,
+                      value);
+}
+
+struct RunEndEncondingState : public KernelState {
+  explicit RunEndEncondingState(std::shared_ptr<DataType> run_end_type)
+      : run_end_type{std::move(run_end_type)} {}
+
+  ~RunEndEncondingState() override = default;
+
+  std::shared_ptr<DataType> run_end_type;
+};
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodingLoop {
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+ private:
+  const int64_t input_length_;
+  const int64_t input_offset_;
+
+  const uint8_t* input_validity_;
+  const void* input_values_;
+
+  // Needed only by WriteEncodedRuns()
+  uint8_t* output_validity_;
+  void* output_values_;
+  RunEndCType* output_run_ends_;
+
+ public:
+  RunEndEncodingLoop(int64_t input_length, int64_t input_offset,
+                     const uint8_t* input_validity, const void* input_values,
+                     uint8_t* output_validity = NULLPTR, void* output_values = NULLPTR,
+                     RunEndCType* output_run_ends = NULLPTR)
+      : input_length_(input_length),
+        input_offset_(input_offset),
+        input_validity_(input_validity),
+        input_values_(input_values),
+        output_validity_(output_validity),
+        output_values_(output_values),
+        output_run_ends_(output_run_ends) {
+    DCHECK_GT(input_length, 0);
+  }
+
+ private:
+  [[nodiscard]] inline bool ReadValue(CType* out, int64_t read_offset) const {
+    return ReadValueImpl<ValueType, has_validity_buffer>{}.ReadValue(
+        input_validity_, input_values_, out, read_offset);
+  }
+
+  inline void WriteValue(int64_t write_offset, bool valid, CType value) {
+    WriteValueImpl<ValueType, has_validity_buffer>{}.WriteValue(
+        output_validity_, output_values_, write_offset, valid, value);
+  }
+
+ public:
+  /// \brief Give a pass over the input data and count the number of runs
+  ///
+  /// \return a pair with the number of non-null run values and total number of runs
+  ARROW_NOINLINE std::pair<int64_t, int64_t> CountNumberOfRuns() const {
+    int64_t read_offset = input_offset_;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    int64_t num_valid_runs = current_run_valid ? 1 : 0;
+    int64_t num_output_runs = 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Open the new run
+        current_run = value;
+        current_run_valid = valid;
+        // Count the new run
+        num_output_runs += 1;
+        num_valid_runs += valid ? 1 : 0;
+      }
+    }
+    return std::make_pair(num_valid_runs, num_output_runs);
+  }
+
+  ARROW_NOINLINE int64_t WriteEncodedRuns() {
+    DCHECK(output_values_);
+    DCHECK(output_run_ends_);
+    int64_t read_offset = input_offset_;
+    int64_t write_offset = 0;
+    CType current_run;
+    bool current_run_valid = ReadValue(&current_run, read_offset);
+    read_offset += 1;
+    for (; read_offset < input_offset_ + input_length_; read_offset += 1) {
+      CType value;
+      const bool valid = ReadValue(&value, read_offset);
+
+      const bool open_new_run = valid != current_run_valid || value != current_run;
+      if (open_new_run) {
+        // Close the current run first by writing it out
+        WriteValue(write_offset, current_run_valid, current_run);
+        const int64_t run_end = read_offset - input_offset_;
+        output_run_ends_[write_offset] = static_cast<RunEndCType>(run_end);
+        write_offset += 1;
+        // Open the new run
+        current_run_valid = valid;
+        current_run = value;
+      }
+    }
+    WriteValue(write_offset, current_run_valid, current_run);
+    DCHECK_EQ(input_length_, read_offset - input_offset_);
+    output_run_ends_[write_offset] = static_cast<RunEndCType>(input_length_);
+    return write_offset + 1;
+  }
+};
+
+template <typename RunEndType>
+Status ValidateRunEndType(int64_t input_length) {
+  using RunEndCType = typename RunEndType::c_type;
+  constexpr int64_t kRunEndMax = std::numeric_limits<RunEndCType>::max();
+  if (input_length > kRunEndMax) {
+    return Status::Invalid(
+        "Cannot run-end encode Arrays with more elements than the "
+        "run end type can hold: ",
+        kRunEndMax);
+  }
+  return Status::OK();
+}
+
+template <typename RunEndType, typename ValueType, bool has_validity_buffer>
+class RunEndEncodeImpl {
+ private:
+  KernelContext* ctx_;
+  const ArraySpan& input_array_;
+  ExecResult* output_;
+
+ public:
+  using RunEndCType = typename RunEndType::c_type;
+  using CType = typename ValueType::c_type;
+
+  RunEndEncodeImpl(KernelContext* ctx, const ArraySpan& input_array, ExecResult* out)
+      : ctx_{ctx}, input_array_{input_array}, output_{out} {}
+
+  Status Exec() {
+    const int64_t input_length = input_array_.length;
+    const int64_t input_offset = input_array_.offset;
+    const auto* input_validity = input_array_.buffers[0].data;
+    const auto* input_values = input_array_.buffers[1].data;
+
+    // First pass: count the number of runs
+    int64_t num_valid_runs = 0;
+    int64_t num_output_runs = 0;
+    if (input_length > 0) {
+      RETURN_NOT_OK(ValidateRunEndType<RunEndType>(input_length));
+
+      RunEndEncodingLoop<RunEndType, ValueType, has_validity_buffer> counting_loop(
+          input_array_.length, input_array_.offset, input_validity, input_values);
+      std::tie(num_valid_runs, num_output_runs) = counting_loop.CountNumberOfRuns();
+    }
+
+    // Allocate the output array data
+    std::shared_ptr<ArrayData> output_array_data;
+    int64_t validity_buffer_size = 0;  // in bytes
+    {
+      ARROW_ASSIGN_OR_RAISE(auto run_ends_buffer,
+                            AllocateBuffer(num_output_runs * RunEndType().bit_width(),
+                                           ctx_->memory_pool()));
+      std::shared_ptr<Buffer> validity_buffer = NULLPTR;
+      if constexpr (has_validity_buffer) {
+        validity_buffer_size = bit_util::BytesForBits(num_output_runs);
+        ARROW_ASSIGN_OR_RAISE(validity_buffer,
+                              AllocateBuffer(validity_buffer_size, ctx_->memory_pool()));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                            AllocateBuffer(bit_util::BytesForBits(
+                                               num_output_runs * ValueType().bit_width()),

Review Comment:
   That's true in the case above, but `ValueType` can be `BooleanType` which fits in a single bit.



-- 
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] felipecrv commented on pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on PR #34195:
URL: https://github.com/apache/arrow/pull/34195#issuecomment-1459008357

   @westonpace @zeroshade this is now ready for final review. I've addressed all of @lidavidm's comments.


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

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

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


[GitHub] [arrow] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1128334346


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));
+    ARROW_CHECK_OK(builder->AppendArraySlice(ArraySpan(*child), 0, child->length));
+    array->child_data[1] = builder->Finish().ValueOrDie()->Slice(offset)->data();
+  }
+};
+
+TEST_P(TestRunEndEncodeDecode, EncodeDecodeArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  auto run_ends_array = MakeArray(encoded->child_data[0]);
+  auto values_array = MakeArray(encoded->child_data[1]);
+  ASSERT_OK(MakeArray(encoded)->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*ArrayFromJSON(run_ends_type, data.expected_run_ends_json),
+                      *run_ends_array);
+  ASSERT_ARRAYS_EQUAL(*values_array, *data.expected_values);
+  ASSERT_EQ(encoded->buffers.size(), 1);
+  ASSERT_EQ(encoded->buffers[0], NULLPTR);
+  ASSERT_EQ(encoded->child_data.size(), 2);
+  ASSERT_EQ(run_ends_array->data()->buffers[0], NULLPTR);
+  ASSERT_EQ(run_ends_array->length(), data.expected_values->length());
+  ASSERT_EQ(run_ends_array->offset(), 0);
+  ASSERT_EQ(encoded->length, data.input->length());
+  ASSERT_EQ(encoded->offset, 0);
+  ASSERT_EQ(*encoded->type, RunEndEncodedType(run_ends_type, data.input->type()));
+  ASSERT_EQ(encoded->null_count, 0);
+
+  ASSERT_OK_AND_ASSIGN(Datum decoded_datum, RunEndDecode(encoded));
+  auto decoded = decoded_datum.make_array();
+  ASSERT_OK(decoded->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*decoded, *data.input);
+}
+
+// Encoding an input with an offset results in a completely new encoded array without an
+// offset. This means The EncodeDecodeArray test will never actually decode an array
+// with an offset, even though we have inputs with offsets. This test slices one element
+// off the encoded array and decodes that.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffset) {
+  auto [data, run_ends_type] = GetParam();
+  if (data.input->length() == 0) {
+    return;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first,
+                       RunEndDecode(encoded->Slice(1, encoded->length - 1)));
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_last,
+                       RunEndDecode(encoded->Slice(0, encoded->length - 1)));
+  auto array_without_first = datum_without_first.make_array();
+  auto array_without_last = datum_without_last.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_OK(array_without_last->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input->Slice(1));
+  ASSERT_ARRAYS_EQUAL(*array_without_last,
+                      *data.input->Slice(0, data.input->length() - 1));
+}
+
+// This test creates an run-end encoded array with an offset in the child array, which
+// removes the first run in the test data.
+TEST_P(TestRunEndEncodeDecode, DecodeWithOffsetInChildArray) {
+  auto [data, run_ends_type] = GetParam();
+
+  ASSERT_OK_AND_ASSIGN(Datum encoded_datum,
+                       RunEndEncode(data.input, RunEndEncodeOptions(run_ends_type)));
+
+  auto encoded = encoded_datum.array();
+  this->AddArtificialOffsetInChildArray(encoded.get(), 100);
+  ASSERT_OK_AND_ASSIGN(Datum datum_without_first, RunEndDecode(encoded));
+  auto array_without_first = datum_without_first.make_array();
+  ASSERT_OK(array_without_first->ValidateFull());
+  ASSERT_ARRAYS_EQUAL(*array_without_first, *data.input);
+}
+
+INSTANTIATE_TEST_SUITE_P(

Review Comment:
   Well, vector functions don't really deal with scalars, so scalars are not needed. cc @westonpace
   
   I'm looking into chunked arrays now.



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1126917285


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;
+};
+
+}  // namespace
+
+class TestRunEndEncodeDecode : public ::testing::TestWithParam<
+                                   std::tuple<REETestData, std::shared_ptr<DataType>>> {
+ public:
+  void AddArtificialOffsetInChildArray(ArrayData* array, int64_t offset) {
+    auto& child = array->child_data[1];
+    auto builder = MakeBuilder(child->type).ValueOrDie();
+    ARROW_CHECK_OK(builder->AppendNulls(offset));

Review Comment:
   Thank you for the tip. Done.



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

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

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


[GitHub] [arrow] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1126913376


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode_test.cc:
##########
@@ -0,0 +1,211 @@
+// 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 <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+
+namespace {
+
+struct REETestData {
+  static REETestData JSON(std::shared_ptr<DataType> data_type, std::string input_json,
+                          std::string expected_values_json,
+                          std::string expected_run_ends_json, int64_t input_offset = 0) {
+    auto input_array = ArrayFromJSON(data_type, input_json);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    result.expected_values = ArrayFromJSON(data_type, expected_values_json);
+    result.expected_run_ends_json = std::move(expected_run_ends_json);
+    result.string = input_json;
+    return result;
+  }
+
+  static REETestData NullArray(int64_t input_length, int64_t input_offset = 0) {
+    auto input_array = std::make_shared<arrow::NullArray>(input_length);
+    REETestData result;
+    result.input = input_array->Slice(input_offset);
+    const int64_t input_slice_length = result.input->length();
+    result.expected_values =
+        std::make_shared<arrow::NullArray>(input_slice_length > 0 ? 1 : 0);
+    result.expected_run_ends_json =
+        input_slice_length > 0 ? "[" + std::to_string(input_slice_length) + "]" : "[]";
+    result.string = "[null * " + std::to_string(input_slice_length) + "]";
+    return result;
+  }
+
+  template <typename ArrowType>
+  static REETestData TypeMinMaxNull() {
+    using CType = typename ArrowType::c_type;
+    REETestData result;
+    NumericBuilder<ArrowType> builder;
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::min()));
+    ARROW_EXPECT_OK(builder.AppendNull());
+    ARROW_EXPECT_OK(builder.Append(std::numeric_limits<CType>::max()));
+    result.input = builder.Finish().ValueOrDie();
+    result.expected_values = result.input;
+    result.expected_run_ends_json = "[1, 2, 3]";
+    result.string = "Type min, max, & null values";
+    return result;
+  }
+
+  std::shared_ptr<Array> input;
+  std::shared_ptr<Array> expected_values;
+  std::string expected_run_ends_json;
+  // only used for gtest output
+  std::string string;

Review Comment:
   Renamed to `description`.



-- 
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] zeroshade commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1119510831


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,652 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+

Review Comment:
   that's perfectly fine, was just curious as I didn't see a TODO or other comment when I was skimming through the PR and wanted to make sure that I understood the code correctly that there wasn't an implementation for binary types



-- 
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] felipecrv commented on a diff in pull request #34195: GH-32105: [C++] Encode and decode Run-End Encoded vectors

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on code in PR #34195:
URL: https://github.com/apache/arrow/pull/34195#discussion_r1120135574


##########
cpp/src/arrow/compute/kernels/vector_run_end_encode.cc:
##########
@@ -0,0 +1,652 @@
+// 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 <utility>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/ree_util.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+template <typename ArrowType, bool has_validity_buffer>
+struct ReadValueImpl {
+  using CType = typename ArrowType::c_type;
+
+  [[nodiscard]] bool ReadValue(const uint8_t* input_validity, const void* input_values,
+                               CType* out, int64_t read_offset) const {
+    bool valid = true;
+    if constexpr (has_validity_buffer) {
+      valid = bit_util::GetBit(input_validity, read_offset);
+    }
+    if (valid) {
+      *out = (reinterpret_cast<const CType*>(input_values))[read_offset];
+    }
+    return valid;
+  }
+};
+
+template <>
+bool ReadValueImpl<BooleanType, true>::ReadValue(const uint8_t* input_validity,
+                                                 const void* input_values, CType* out,
+                                                 int64_t read_offset) const {
+  const bool valid = bit_util::GetBit(input_validity, read_offset);
+  *out = valid &&
+         bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return valid;
+}
+
+template <>
+bool ReadValueImpl<BooleanType, false>::ReadValue(const uint8_t* input_validity,
+                                                  const void* input_values, CType* out,
+                                                  int64_t read_offset) const {
+  *out = bit_util::GetBit(reinterpret_cast<const uint8_t*>(input_values), read_offset);
+  return true;
+}
+

Review Comment:
   Oh, right. Added a TODO mentioning the newly created issue.



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