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

[GitHub] [arrow] gringasalpastor opened a new pull request, #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   *Are these changes tested?*
   
   The following tests are provided:
   - basic usage
   - const ranges
   - custom struct accessor
   - usage with `std::variant`
   


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

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

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


[GitHub] [arrow] gringasalpastor commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,
+    DataPointConvertor&& data_point_convertor,
+    RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor()) {
+  const std::size_t batch_size = 1024;

Review Comment:
   sure, will update to make it an parameter



-- 
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] wjones127 commented on pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   @gringasalpastor There seems to be one CI failure that is legitimate. Could you address that?
   
   ```
   /arrow/cpp/src/arrow/util/rows_to_batches.h:107: error: The following parameters of arrow::util::RowsToBatches(const std::shared_ptr< Schema > &schema, const Range &rows, DataPointConvertor &&data_point_convertor, RowAccessor &&row_accessor=detail::MakeDefaultRowAccessor(), MemoryPool *pool=default_memory_pool(), const std::size_t batch_size=1024) are not documented:
     parameter 'pool'
     parameter 'batch_size' (warning treated as error, aborting now)
   1
   ```


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

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

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


[GitHub] [arrow] gringasalpastor commented on pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   @wjones127 If you could take a look, thanks.


-- 
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] wjones127 merged pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


-- 
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] wjones127 commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches_test.cc:
##########
@@ -0,0 +1,120 @@
+// 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 <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_binary.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/scalar.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/rows_to_batches.h"
+
+namespace arrow::util {
+
+const auto kTestSchema = schema(
+    {field("field_1", int64()), field("field_2", int64()), field("field_3", int64())});
+
+auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+  return static_cast<Int64Builder&>(array_builder).Append(value);
+};
+
+bool CompareJson(const arrow::Table& arrow_table, const std::string& json,
+                 const std::string& field_name) {
+  const auto col = arrow_table.GetColumnByName(field_name);
+  return arrow::ChunkedArrayFromJSON(col->type(), {json})->Equals(col);
+};

Review Comment:
   ```suggestion
   }
   ```



-- 
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] gringasalpastor commented on pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   > Just two minor changes, then I think this is ready to go.
   
   Committed the last 2 suggested changes, let me know if there is anything else you would like to see updated @wjones127 


-- 
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 #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   :warning: GitHub issue #34056 **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] gringasalpastor commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,

Review Comment:
   Good question. The intention was to block the client from passing a temporary (`const T&&`). `reference_wrapper ` does this by using `= delete` on that constructor. A temporary doesn't make sense because the lambda will be called after the end of the function call.
   
   After more investigation, I found a stackoverflow showing that you can use `= delete`  on free functions. I am going to try to do that so can can just pass a `const T&`
   
   https://stackoverflow.com/questions/28739974/disallow-passing-of-rvalue-reference-to-a-function



-- 
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] wjones127 commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,
+    DataPointConvertor&& data_point_convertor,
+    RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor()) {

Review Comment:
   We try to always allow the user to pass down their own memory pool:
   
   ```suggestion
       RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor(),
       MemoryPool* pool = default_memory_pool()) {
   ```



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,
+    DataPointConvertor&& data_point_convertor,
+    RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor()) {
+  const std::size_t batch_size = 1024;

Review Comment:
   Also perhaps this should be a parameter with a default?



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,
+    DataPointConvertor&& data_point_convertor,
+    RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor()) {
+  const std::size_t batch_size = 1024;
+  auto make_next_batch =
+      [rows_ittr = std::begin(rows.get()), rows_ittr_end = std::end(rows.get()),
+       schema = schema, row_accessor = std::forward<RowAccessor>(row_accessor),
+       data_point_convertor = std::forward<DataPointConvertor>(
+           data_point_convertor)]() mutable -> Result<std::shared_ptr<RecordBatch>> {
+    if (rows_ittr == rows_ittr_end) return NULLPTR;
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto record_batch_builder,
+        RecordBatchBuilder::Make(schema, default_memory_pool(), batch_size));
+
+    for (size_t i = 0; i < batch_size and (rows_ittr != rows_ittr_end);
+         i++, std::advance(rows_ittr, 1)) {
+      size_t col_index = 0;

Review Comment:
   This is int in `GetField()`, so should match that.
   ```suggestion
         int col_index = 0;
   ```



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`

Review Comment:
   This is obvious in most cases, but worth saying for some others: it needs to return the data points in the proper order.
   
   Also: "otter `rows`"?
   
   ```suggestion
   /// \param[in] row_accessor - In the common case where the value of each row iterated
   /// over is it's self also directly iterable, the client can just use the default.
   /// The provided callable must take the values of the otter `rows` range and return a
   /// `std::reference_wrapper<Range>` to the data points in a given row. The data points
   /// must be in order of their corresponding fields in the schema.
   /// see: /ref `MakeDefaultRowAccessor`
   ```



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,
+    DataPointConvertor&& data_point_convertor,
+    RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor()) {
+  const std::size_t batch_size = 1024;
+  auto make_next_batch =
+      [rows_ittr = std::begin(rows.get()), rows_ittr_end = std::end(rows.get()),
+       schema = schema, row_accessor = std::forward<RowAccessor>(row_accessor),
+       data_point_convertor = std::forward<DataPointConvertor>(
+           data_point_convertor)]() mutable -> Result<std::shared_ptr<RecordBatch>> {
+    if (rows_ittr == rows_ittr_end) return NULLPTR;
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto record_batch_builder,
+        RecordBatchBuilder::Make(schema, default_memory_pool(), batch_size));

Review Comment:
   ```suggestion
           RecordBatchBuilder::Make(schema, pool, batch_size));
   ```



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,

Review Comment:
   `std::reference_wrapper` is new to me. Is there a reason you use that rather than taking `const Range&`? It seems like this requires that users wrap their types with `std::ref` or `std::cref`.



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,
+    DataPointConvertor&& data_point_convertor,
+    RowAccessor&& row_accessor = detail::MakeDefaultRowAccessor()) {
+  const std::size_t batch_size = 1024;
+  auto make_next_batch =
+      [rows_ittr = std::begin(rows.get()), rows_ittr_end = std::end(rows.get()),
+       schema = schema, row_accessor = std::forward<RowAccessor>(row_accessor),
+       data_point_convertor = std::forward<DataPointConvertor>(
+           data_point_convertor)]() mutable -> Result<std::shared_ptr<RecordBatch>> {
+    if (rows_ittr == rows_ittr_end) return NULLPTR;
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto record_batch_builder,
+        RecordBatchBuilder::Make(schema, default_memory_pool(), batch_size));
+
+    for (size_t i = 0; i < batch_size and (rows_ittr != rows_ittr_end);

Review Comment:
   In this project we always prefer `&&`:
   ```suggestion
       for (size_t i = 0; i < batch_size && (rows_ittr != rows_ittr_end);
   ```



-- 
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] wjones127 commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches_test.cc:
##########
@@ -0,0 +1,120 @@
+// 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 <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_binary.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/scalar.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/rows_to_batches.h"
+
+namespace arrow::util {
+
+const auto kTestSchema = schema(
+    {field("field_1", int64()), field("field_2", int64()), field("field_3", int64())});
+
+auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+  return static_cast<Int64Builder&>(array_builder).Append(value);
+};
+
+bool CompareJson(const arrow::Table& arrow_table, const std::string& json,
+                 const std::string& field_name) {
+  const auto col = arrow_table.GetColumnByName(field_name);
+  return arrow::ChunkedArrayFromJSON(col->type(), {json})->Equals(col);
+}
+
+TEST(RowsToBatches, BasicUsage) {
+  std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+  auto batches = RowsToBatches(kTestSchema, data, IntConvertor).ValueOrDie();
+  auto table = batches->ToTable().ValueOrDie();
+
+  EXPECT_TRUE(CompareJson(*table, R"([1, 5])", "field_1"));
+  EXPECT_TRUE(CompareJson(*table, R"([2, 6])", "field_2"));
+  EXPECT_TRUE(CompareJson(*table, R"([4, 7])", "field_3"));
+}
+
+TEST(RowsToBatches, ConstRange) {
+  const std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+  auto batches = RowsToBatches(kTestSchema, data, IntConvertor).ValueOrDie();
+  auto table = batches->ToTable().ValueOrDie();
+
+  EXPECT_TRUE(CompareJson(*table, R"([1, 5])", "field_1"));
+  EXPECT_TRUE(CompareJson(*table, R"([2, 6])", "field_2"));
+  EXPECT_TRUE(CompareJson(*table, R"([4, 7])", "field_3"));
+}
+
+TEST(RowsToBatches, StructAccessor) {
+  struct TestStruct {
+    std::vector<int> values;
+  };
+  std::vector<TestStruct> data = {TestStruct{{1, 2, 4}}, TestStruct{{5, 6, 7}}};
+
+  auto accessor =
+      [](const TestStruct& s) -> Result<std::reference_wrapper<const std::vector<int>>> {
+    return std::cref(s.values);
+  };
+
+  auto batches = RowsToBatches(kTestSchema, data, IntConvertor, accessor).ValueOrDie();
+
+  auto table = batches->ToTable().ValueOrDie();
+
+  EXPECT_TRUE(CompareJson(*table, R"([1, 5])", "field_1"));
+  EXPECT_TRUE(CompareJson(*table, R"([2, 6])", "field_2"));
+  EXPECT_TRUE(CompareJson(*table, R"([4, 7])", "field_3"));
+
+  // Test accessor that returns by value instead of using `std::reference_wrapper`
+  auto accessor_by_value = [](const TestStruct& s) -> Result<std::set<int>> {
+    return std::set(std::begin(s.values), std::end(s.values));
+  };
+  auto batches_by_value =
+      RowsToBatches(kTestSchema, data, IntConvertor, accessor_by_value).ValueOrDie();
+
+  auto table_by_value = batches_by_value->ToTable().ValueOrDie();
+
+  EXPECT_TRUE(CompareJson(*table_by_value, R"([1, 5])", "field_1"));
+  EXPECT_TRUE(CompareJson(*table_by_value, R"([2, 6])", "field_2"));
+  EXPECT_TRUE(CompareJson(*table_by_value, R"([4, 7])", "field_3"));
+}
+
+TEST(RowsToBatches, Variant) {
+  auto VariantConvertor = [](ArrayBuilder& array_builder,
+                             const std::variant<int, std::string>& value) {
+    if (std::holds_alternative<int>(value))
+      return dynamic_cast<Int64Builder&>(array_builder).Append(std::get<int>(value));
+    else
+      return dynamic_cast<arrow::StringBuilder&>(array_builder)
+          .Append(std::get<std::string>(value).c_str(),
+                  std::get<std::string>(value).length());

Review Comment:
   ```suggestion
             .Append(std::get<std::string>(value));
   ```



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,159 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /

Review Comment:
   ```suggestion
   // Meta-function to check if a type `T` is a range (iterable using `std::begin()` /
   ```



-- 
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] gringasalpastor closed pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

Posted by "gringasalpastor (via GitHub)" <gi...@apache.org>.
gringasalpastor closed pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or  an `arrow::Table`
URL: https://github.com/apache/arrow/pull/34057


-- 
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 #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   Benchmark runs are scheduled for baseline = 92d91f530f581005fc9f8c5a5493f2eee2fa9668 and contender = 7828165f185b5ea2a3e76c06f9d4ba44263fd6dc. 7828165f185b5ea2a3e76c06f9d4ba44263fd6dc 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/112cccc8d94d46ab9069d06236ea3663...257d54ea70454e029608558e91646b61/)
   [Failed :arrow_down:0.79% :arrow_up:0.0%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/642054a948dc4125b6be07887c1fcdd6...5cea5c0d01e74a1a86c8bc6aa8049805/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/4dc84a95404742509cda452c74562787...2b990b82fe9a462b936030d141b1bd5d/)
   [Finished :arrow_down:0.83% :arrow_up:0.06%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/f028091c7a0446f6a99814e82b828f31...cb6bd007d009406fa11dcf78d1a82347/)
   Buildkite builds:
   [Finished] [`7828165f` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2410)
   [Failed] [`7828165f` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2440)
   [Finished] [`7828165f` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2408)
   [Finished] [`7828165f` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2432)
   [Finished] [`92d91f53` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2409)
   [Finished] [`92d91f53` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2439)
   [Finished] [`92d91f53` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2407)
   [Finished] [`92d91f53` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2431)
   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] gringasalpastor commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,

Review Comment:
   Updated PR to use `const &&` and delete overload for `const T&&`



##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,

Review Comment:
   Updated PR to use `const &` and delete overload for `const T&&`



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

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

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


[GitHub] [arrow] github-actions[bot] commented on pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   * Closes: #34056


-- 
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] gringasalpastor commented on a diff in pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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


##########
cpp/src/arrow/util/rows_to_batches.h:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/table_builder.h"
+#include "arrow/util/iterator.h"
+
+#include <type_traits>
+
+namespace arrow::util {
+
+namespace detail {
+
+// Default identity function row accessor. Used to for the common case where the value
+// of each row iterated over is it's self also directly iterable.
+[[nodiscard]] constexpr inline auto MakeDefaultRowAccessor() {
+  return [](auto& x) -> Result<decltype(std::ref(x))> { return std::ref(x); };
+}
+
+// Meta-funciton to check if a type `T` is a range (iterable using `std::begin()` /
+// `std::end()`). `is_range<T>::value` will be false if `T` is not a valid range.
+template <typename T, typename = void>
+struct is_range : std::false_type {};
+
+template <typename T>
+struct is_range<T, std::void_t<decltype(std::begin(std::declval<T>())),
+                               decltype(std::end(std::declval<T>()))>> : std::true_type {
+};
+
+}  // namespace detail
+
+/// \brief Utility function for converting any row-based structure into an
+/// `arrow::RecordBatchReader` (this can be easily converted to an `arrow::Table` using
+/// `arrow::RecordBatchReader::ToTable()`).
+///
+/// Examples of supported types:
+/// - `std::vector<std::vector<std::variant<int, bsl::string>>>`
+/// - `std::vector<MyRowStruct>`
+
+/// If `rows` (client’s row-based structure) is not a valid C++ range, the client will
+/// need to either make it iterable, or make an adapter/wrapper that is a valid C++
+/// range.
+
+/// The client must provide a `DataPointConvertor` callable type that will convert the
+/// structure’s data points into the corresponding arrow types.
+
+/// Complex nested rows can be supported by providing a custom `row_accessor` instead
+/// of the default.
+
+/// Example usage:
+/// \code{.cpp}
+/// auto IntConvertor = [](ArrayBuilder& array_builder, int value) {
+///  return static_cast<Int64Builder&>(array_builder).Append(value);
+/// };
+/// std::vector<std::vector<int>> data = {{1, 2, 4}, {5, 6, 7}};
+/// auto batches = RowsToBatches(kTestSchema, std::ref(data), IntConvertor);
+/// \endcode
+
+/// \param[in] schema - the schema to be used in the `RecordBatchReader`
+
+/// \param[in] rows - iterable row-based structure that will be converted to arrow
+/// batches
+
+/// \param[in] data_point_convertor - client provided callable type that will convert
+/// the structure’s data points into the corresponding arrow types. The convertor must
+/// return an error `Status` if an error happens during conversion.
+
+/// \param[in] row_accessor - In the common case where the value of each row iterated
+/// over is it's self also directly iterable, the client can just use the default.
+/// the provided callable must take the values of the otter `rows` range and return a
+/// `std::reference_wrapper<Range>` to the data points in a given row.
+/// see: /ref `MakeDefaultRowAccessor`
+
+/// \return `Result<std::shared_ptr<RecordBatchReader>>>` result will be a
+/// `std::shared_ptr<RecordBatchReader>>` if not errors occurred, else an error status.
+template <class Range, class DataPointConvertor,
+          class RowAccessor = decltype(detail::MakeDefaultRowAccessor())>
+[[nodiscard]] typename std::enable_if_t<detail::is_range<Range>::value,
+                                        Result<std::shared_ptr<RecordBatchReader>>>
+/* Result<std::shared_ptr<RecordBatchReader>>> */ RowsToBatches(
+    const std::shared_ptr<Schema>& schema, std::reference_wrapper<Range> rows,

Review Comment:
   Good question. The intention was to block the client from passing a temporary (`const T&&`). `reference_wrapper ` does this by using `= delete` on that constructor. A temporary doesn't make sense because the lambda will be called after the end of the function call.
   
   After more investigation, I found a stackoverflow showing that you can use `= delete`  on free functions. I am going to try to do that so can can just pass a `const T&`



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

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

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


[GitHub] [arrow] gringasalpastor commented on pull request #34057: GH-34056: [C++] Add Utility function to simplify converting any row-based structure into an `arrow::RecordBatchReader` or an `arrow::Table`

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

   > @gringasalpastor There seems to be one CI failure that is legitimate. Could you address that?
   > 
   > ```
   > /arrow/cpp/src/arrow/util/rows_to_batches.h:107: error: The following parameters of arrow::util::RowsToBatches(const std::shared_ptr< Schema > &schema, const Range &rows, DataPointConvertor &&data_point_convertor, RowAccessor &&row_accessor=detail::MakeDefaultRowAccessor(), MemoryPool *pool=default_memory_pool(), const std::size_t batch_size=1024) are not documented:
   >   parameter 'pool'
   >   parameter 'batch_size' (warning treated as error, aborting now)
   > 1
   > ```
   
   @wjones127 Thanks, I just pushed a new commit to add documentation for these. 


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